This is an automated email from the ASF dual-hosted git repository.
dockerzhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/inlong.git
The following commit(s) were added to refs/heads/master by this push:
new f7e3e3cd4e [INLONG-9692][Sort] Support rowdata way of sort InLong
message tlog-csv format (#9693)
f7e3e3cd4e is described below
commit f7e3e3cd4e8235fc750073a2b7f90d3b2d483607
Author: baomingyu <[email protected]>
AuthorDate: Mon Feb 19 15:09:35 2024 +0800
[INLONG-9692][Sort] Support rowdata way of sort InLong message tlog-csv
format (#9693)
---
.../format-inlongmsg-rowdata-tlogcsv/pom.xml | 89 ++++++
.../InLongMsgTlogCsvDeserializationSchema.java | 129 ++++++++
.../InLongMsgTlogCsvFormatDeserializer.java | 280 +++++++++++++++++
.../InLongMsgTlogCsvFormatFactory.java | 189 ++++++++++++
.../inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java | 170 +++++++++++
.../org.apache.flink.table.factories.Factory | 16 +
.../InLongMsgTlogCsvFormatDeserializerTest.java | 337 +++++++++++++++++++++
.../InLongMsgTlogCsvFormatFactoryTest.java | 127 ++++++++
.../src/test/resources/log4j-test.properties | 27 ++
inlong-sort/sort-formats/format-rowdata/pom.xml | 1 +
10 files changed, 1365 insertions(+)
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/pom.xml
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/pom.xml
new file mode 100644
index 0000000000..d33fc06555
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/pom.xml
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.inlong</groupId>
+ <artifactId>format-rowdata</artifactId>
+ <version>1.12.0-SNAPSHOT</version>
+ </parent>
+ <artifactId>sort-format-inlongmsg-rowdata-tlogcsv</artifactId>
+ <packaging>jar</packaging>
+ <name>Apache InLong - Sort Format-InLongMsg-RowData-TLogCSV</name>
+
+ <properties>
+
<inlong.root.dir>${project.parent.parent.parent.parent.basedir}</inlong.root.dir>
+ </properties>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.inlong</groupId>
+ <artifactId>sort-format-inlongmsg-rowdata-base</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-table-common</artifactId>
+ <version>${flink.version}</version>
+ <scope>provided</scope>
+ </dependency>
+
+ <!-- test dependencies -->
+ <!-- CSV table descriptor testing -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-table-common</artifactId>
+ <version>${flink.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- CSV RowData (de)serialization schema testing -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+
+ <!-- CSV rowData encoder/intputformat testing -->
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-test-utils</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>log4j-slf4j-impl</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+ <version>${flink.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvDeserializationSchema.java
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvDeserializationSchema.java
new file mode 100644
index 0000000000..3fb30ccecd
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvDeserializationSchema.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.inlong.sort.formats.inlongmsgtlogcsv;
+
+import org.apache.inlong.sort.formats.common.RowFormatInfo;
+import
org.apache.inlong.sort.formats.inlongmsg.AbstractInLongMsgDeserializationSchema;
+import
org.apache.inlong.sort.formats.inlongmsg.AbstractInLongMsgFormatDeserializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import java.util.Collections;
+import java.util.List;
+
+import static
org.apache.inlong.sort.formats.base.TableFormatConstants.DEFAULT_CHARSET;
+import static
org.apache.inlong.sort.formats.base.TableFormatConstants.DEFAULT_DELIMITER;
+import static
org.apache.inlong.sort.formats.base.TableFormatConstants.DEFAULT_ESCAPE_CHARACTER;
+import static
org.apache.inlong.sort.formats.base.TableFormatConstants.DEFAULT_NULL_LITERAL;
+import static
org.apache.inlong.sort.formats.base.TableFormatConstants.DEFAULT_QUOTE_CHARACTER;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.DEFAULT_ATTRIBUTES_FIELD_NAME;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.DEFAULT_TIME_FIELD_NAME;
+
+/**
+ * Deserialization schema from InLongMsg TlogCsv to Flink Table & SQL
internal data structures.
+ */
+public class InLongMsgTlogCsvDeserializationSchema extends
AbstractInLongMsgDeserializationSchema {
+
+ public
InLongMsgTlogCsvDeserializationSchema(AbstractInLongMsgFormatDeserializer
formatDeserializer) {
+ super(formatDeserializer);
+ }
+
+ /**
+ * A builder for creating a {@link InLongMsgTlogCsvDeserializationSchema}.
+ */
+ @PublicEvolving
+ public static class Builder {
+
+ private final RowFormatInfo rowFormatInfo;
+
+ private String timeFieldName = DEFAULT_TIME_FIELD_NAME;
+ private String attributesFieldName = DEFAULT_ATTRIBUTES_FIELD_NAME;
+ private String charset = DEFAULT_CHARSET;
+ private Character delimiter = DEFAULT_DELIMITER;
+ private Character escapeChar = DEFAULT_ESCAPE_CHARACTER;
+ private Character quoteChar = DEFAULT_QUOTE_CHARACTER;
+ private String nullLiteral = DEFAULT_NULL_LITERAL;
+ private boolean ignoreErrors = false;
+ private List<String> metadataKeys = Collections.emptyList();
+
+ protected Builder(RowFormatInfo rowFormatInfo) {
+ this.rowFormatInfo = rowFormatInfo;
+ }
+
+ public Builder setTimeFieldName(String timeFieldName) {
+ this.timeFieldName = timeFieldName;
+ return this;
+ }
+
+ public Builder setAttributesFieldName(String attributesFieldName) {
+ this.attributesFieldName = attributesFieldName;
+ return this;
+ }
+
+ public Builder setDelimiter(char delimiter) {
+ this.delimiter = delimiter;
+ return this;
+ }
+
+ public Builder setCharset(String charset) {
+ this.charset = charset;
+ return this;
+ }
+
+ public Builder setEscapeCharacter(char escapeChar) {
+ this.escapeChar = escapeChar;
+ return this;
+ }
+
+ public Builder setQuoteCharacter(char quoteChar) {
+ this.quoteChar = quoteChar;
+ return this;
+ }
+
+ public Builder setNullLiteral(String nullLiteral) {
+ this.nullLiteral = nullLiteral;
+ return this;
+ }
+
+ public Builder setIgnoreErrors(boolean ignoreErrors) {
+ this.ignoreErrors = ignoreErrors;
+ return this;
+ }
+
+ public Builder setMetadataKeys(List<String> metadataKeys) {
+ this.metadataKeys = metadataKeys;
+ return this;
+ }
+
+ public InLongMsgTlogCsvDeserializationSchema build() {
+ AbstractInLongMsgFormatDeserializer formatDeserializer = new
InLongMsgTlogCsvFormatDeserializer(
+ rowFormatInfo,
+ timeFieldName,
+ attributesFieldName,
+ charset,
+ delimiter,
+ escapeChar,
+ quoteChar,
+ nullLiteral,
+ metadataKeys,
+ ignoreErrors);
+
+ return new
InLongMsgTlogCsvDeserializationSchema(formatDeserializer);
+ }
+ }
+}
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializer.java
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializer.java
new file mode 100644
index 0000000000..32f2edb7ca
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializer.java
@@ -0,0 +1,280 @@
+/*
+ * 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.inlong.sort.formats.inlongmsgtlogcsv;
+
+import org.apache.inlong.sort.formats.base.FieldToRowDataConverters;
+import
org.apache.inlong.sort.formats.base.FieldToRowDataConverters.FieldToRowDataConverter;
+import org.apache.inlong.sort.formats.base.TableFormatUtils;
+import org.apache.inlong.sort.formats.base.TextFormatBuilder;
+import org.apache.inlong.sort.formats.common.RowFormatInfo;
+import
org.apache.inlong.sort.formats.inlongmsg.AbstractInLongMsgFormatDeserializer;
+import org.apache.inlong.sort.formats.inlongmsg.FailureHandler;
+import org.apache.inlong.sort.formats.inlongmsg.InLongMsgBody;
+import org.apache.inlong.sort.formats.inlongmsg.InLongMsgHead;
+import org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.types.Row;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import static
org.apache.inlong.sort.formats.base.TableFormatConstants.DEFAULT_DELIMITER;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.DEFAULT_ATTRIBUTES_FIELD_NAME;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.DEFAULT_TIME_FIELD_NAME;
+
+/**
+ * The deserializer for the records in InLongMsgTlogCsv format.
+ */
+public final class InLongMsgTlogCsvFormatDeserializer extends
AbstractInLongMsgFormatDeserializer {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Format information describing the result type.
+ */
+ @Nonnull
+ private final RowFormatInfo rowFormatInfo;
+
+ /**
+ * The name of the time field.
+ */
+ @Nullable
+ private final String timeFieldName;
+
+ /**
+ * The name of the attributes field.
+ */
+ @Nullable
+ private final String attributesFieldName;
+
+ /**
+ * The charset of the text.
+ */
+ private final String charset;
+
+ /**
+ * The delimiter between fields.
+ */
+ @Nonnull
+ private final Character delimiter;
+
+ /**
+ * Escape character. Null if escaping is disabled.
+ */
+ @Nullable
+ private final Character escapeChar;
+
+ /**
+ * Quote character. Null if quoting is disabled.
+ */
+ @Nullable
+ private final Character quoteChar;
+
+ /**
+ * The literal represented null values, default "".
+ */
+ @Nullable
+ private final String nullLiteral;
+
+ private final List<String> metadataKeys;
+
+ private final FieldToRowDataConverter[] converters;
+
+ public InLongMsgTlogCsvFormatDeserializer(
+ @Nonnull RowFormatInfo rowFormatInfo,
+ @Nullable String timeFieldName,
+ @Nullable String attributesFieldName,
+ @Nonnull String charset,
+ @Nonnull Character delimiter,
+ @Nullable Character escapeChar,
+ @Nullable Character quoteChar,
+ @Nullable String nullLiteral,
+ List<String> metadataKeys,
+ @Nonnull Boolean ignoreErrors) {
+ this(
+ rowFormatInfo,
+ timeFieldName,
+ attributesFieldName,
+ charset,
+ delimiter,
+ escapeChar,
+ quoteChar,
+ nullLiteral,
+ metadataKeys,
+ InLongMsgUtils.getDefaultExceptionHandler(ignoreErrors));
+ }
+
+ public InLongMsgTlogCsvFormatDeserializer(
+ @Nonnull RowFormatInfo rowFormatInfo,
+ @Nullable String timeFieldName,
+ @Nullable String attributesFieldName,
+ @Nonnull String charset,
+ @Nonnull Character delimiter,
+ @Nullable Character escapeChar,
+ @Nullable Character quoteChar,
+ @Nullable String nullLiteral,
+ List<String> metadataKeys,
+ @Nonnull FailureHandler failureHandler) {
+ super(failureHandler);
+
+ this.rowFormatInfo = rowFormatInfo;
+ this.timeFieldName = timeFieldName;
+ this.attributesFieldName = attributesFieldName;
+ this.charset = charset;
+ this.delimiter = delimiter;
+ this.escapeChar = escapeChar;
+ this.quoteChar = quoteChar;
+ this.nullLiteral = nullLiteral;
+ this.metadataKeys = metadataKeys;
+
+ converters = Arrays.stream(rowFormatInfo.getFieldFormatInfos())
+ .map(formatInfo -> FieldToRowDataConverters.createConverter(
+ TableFormatUtils.deriveLogicalType(formatInfo)))
+ .toArray(FieldToRowDataConverter[]::new);
+ }
+
+ @Override
+ public TypeInformation<RowData> getProducedType() {
+ return InLongMsgUtils.decorateRowTypeWithNeededHeadFieldsAndMetadata(
+ timeFieldName,
+ attributesFieldName,
+ rowFormatInfo,
+ metadataKeys);
+ }
+
+ @Override
+ protected InLongMsgHead parseHead(String attr) throws Exception {
+ return InLongMsgTlogCsvUtils.parseHead(attr);
+ }
+
+ @Override
+ protected List<InLongMsgBody> parseBodyList(byte[] bytes) throws Exception
{
+ return Collections.singletonList(
+ InLongMsgTlogCsvUtils.parseBody(bytes, charset, delimiter,
escapeChar, quoteChar));
+ }
+
+ @Override
+ protected List<RowData> convertRowDataList(InLongMsgHead head,
InLongMsgBody body) throws Exception {
+ Row dataRow =
+ InLongMsgTlogCsvUtils.deserializeRow(
+ rowFormatInfo,
+ nullLiteral,
+ head.getPredefinedFields(),
+ body.getFields());
+
+ GenericRowData genericRowData = (GenericRowData)
InLongMsgUtils.decorateRowWithNeededHeadFields(
+ timeFieldName,
+ attributesFieldName,
+ head.getTime(),
+ head.getAttributes(),
+ dataRow,
+ converters);
+
+ return
Collections.singletonList(InLongMsgUtils.decorateRowWithMetaData(genericRowData,
head, metadataKeys));
+ }
+
+ /**
+ * The builder for {@link InLongMsgTlogCsvFormatDeserializer}.
+ */
+ public static class Builder extends TextFormatBuilder<Builder> {
+
+ private String timeFieldName = DEFAULT_TIME_FIELD_NAME;
+ private String attributesFieldName = DEFAULT_ATTRIBUTES_FIELD_NAME;
+ private Character delimiter = DEFAULT_DELIMITER;
+ private List<String> metadataKeys = Collections.emptyList();
+
+ public Builder(RowFormatInfo rowFormatInfo) {
+ super(rowFormatInfo);
+ }
+
+ public Builder setTimeFieldName(String timeFieldName) {
+ this.timeFieldName = timeFieldName;
+ return this;
+ }
+
+ public Builder setAttributesFieldName(String attributesFieldName) {
+ this.attributesFieldName = attributesFieldName;
+ return this;
+ }
+
+ public Builder setDelimiter(Character delimiter) {
+ this.delimiter = delimiter;
+ return this;
+ }
+
+ public Builder setMetadataKeys(List<String> metadataKeys) {
+ this.metadataKeys = metadataKeys;
+ return this;
+ }
+
+ public InLongMsgTlogCsvFormatDeserializer build() {
+ return new InLongMsgTlogCsvFormatDeserializer(
+ rowFormatInfo,
+ timeFieldName,
+ attributesFieldName,
+ charset,
+ delimiter,
+ escapeChar,
+ quoteChar,
+ nullLiteral,
+ metadataKeys,
+ ignoreErrors);
+ }
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ if (!super.equals(o)) {
+ return false;
+ }
+
+ InLongMsgTlogCsvFormatDeserializer that =
(InLongMsgTlogCsvFormatDeserializer) o;
+ return rowFormatInfo.equals(that.rowFormatInfo) &&
+ Objects.equals(timeFieldName, that.timeFieldName) &&
+ Objects.equals(attributesFieldName, that.attributesFieldName)
&&
+ Objects.equals(charset, that.charset) &&
+ delimiter.equals(that.delimiter) &&
+ Objects.equals(escapeChar, that.escapeChar) &&
+ Objects.equals(quoteChar, that.quoteChar) &&
+ Objects.equals(nullLiteral, that.nullLiteral) &&
+ Objects.equals(metadataKeys, that.metadataKeys);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(super.hashCode(), rowFormatInfo, timeFieldName,
+ attributesFieldName, charset, delimiter, escapeChar, quoteChar,
+ nullLiteral, metadataKeys);
+ }
+}
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatFactory.java
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatFactory.java
new file mode 100644
index 0000000000..76a9b90941
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatFactory.java
@@ -0,0 +1,189 @@
+/*
+ * 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.inlong.sort.formats.inlongmsgtlogcsv;
+
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.connector.ChangelogMode;
+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.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static
org.apache.inlong.sort.formats.base.TableFormatOptions.ROW_FORMAT_INFO;
+import static
org.apache.inlong.sort.formats.base.TableFormatUtils.deserializeRowFormatInfo;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgOptions.ATTRIBUTE_FIELD_NAME;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgOptions.CHARSET;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgOptions.ESCAPE_CHARACTER;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgOptions.FIELD_DELIMITER;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgOptions.IGNORE_ERRORS;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgOptions.NULL_LITERAL;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgOptions.QUOTE_CHARACTER;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgOptions.TIME_FIELD_NAME;
+
+/**
+ * Table format factory for providing configured instances of
InLongMsgTlogCsv-to-row
+ * serializer and deserializer.
+ */
+public final class InLongMsgTlogCsvFormatFactory implements
DeserializationFormatFactory {
+
+ public static final String IDENTIFIER = "inlong-msg-tlogcsv";
+
+ @Override
+ public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(
+ DynamicTableFactory.Context context, ReadableConfig formatOptions)
{
+ FactoryUtil.validateFactoryOptions(this, formatOptions);
+ validateFormatOptions(formatOptions);
+
+ return new DecodingFormat<DeserializationSchema<RowData>>() {
+
+ @Override
+ public DeserializationSchema<RowData> createRuntimeDecoder(
+ DynamicTableSource.Context context, DataType
producedDataType) {
+
+ InLongMsgTlogCsvDeserializationSchema.Builder builder =
+ new InLongMsgTlogCsvDeserializationSchema.Builder(
+
deserializeRowFormatInfo(formatOptions.get(ROW_FORMAT_INFO)));
+ configureDeserializationSchema(formatOptions, builder);
+ return builder.build();
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode() {
+ return ChangelogMode.insertOnly();
+ }
+ };
+ }
+
+ @Override
+ public String factoryIdentifier() {
+ return IDENTIFIER;
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ return Stream.of(ROW_FORMAT_INFO).collect(Collectors.toSet());
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ Set<ConfigOption<?>> options = new HashSet<>();
+ options.add(TIME_FIELD_NAME);
+ options.add(ATTRIBUTE_FIELD_NAME);
+ options.add(CHARSET);
+ options.add(FIELD_DELIMITER);
+ options.add(QUOTE_CHARACTER);
+ options.add(ESCAPE_CHARACTER);
+ options.add(NULL_LITERAL);
+ options.add(IGNORE_ERRORS);
+ return options;
+ }
+
+ // ------------------------------------------------------------------------
+ // Validation
+ // ------------------------------------------------------------------------
+
+ static void validateFormatOptions(ReadableConfig tableOptions) {
+ // Validate the option value must be a single char.
+ validateCharacterVal(tableOptions, CHARSET);
+ validateCharacterVal(tableOptions, TIME_FIELD_NAME);
+ validateCharacterVal(tableOptions, ATTRIBUTE_FIELD_NAME);
+ validateCharacterVal(tableOptions, FIELD_DELIMITER, true);
+ validateCharacterVal(tableOptions, QUOTE_CHARACTER);
+ validateCharacterVal(tableOptions, ESCAPE_CHARACTER);
+ }
+
+ /**
+ * Validates the option {@code option} value must be a Character.
+ */
+ private static void validateCharacterVal(
+ ReadableConfig tableOptions, ConfigOption<String> option) {
+ validateCharacterVal(tableOptions, option, false);
+ }
+
+ /**
+ * Validates the option {@code option} value must be a Character.
+ *
+ * @param tableOptions the table options
+ * @param option the config option
+ * @param unescape whether to unescape the option value
+ */
+ private static void validateCharacterVal(
+ ReadableConfig tableOptions, ConfigOption<String> option, boolean
unescape) {
+ if (tableOptions.getOptional(option).isPresent()) {
+ final String value =
+ unescape
+ ?
StringEscapeUtils.unescapeJava(tableOptions.get(option))
+ : tableOptions.get(option);
+ if (value.length() != 1) {
+ throw new ValidationException(
+ String.format(
+ "Option '%s.%s' must be a string with single
character, but was: %s",
+ IDENTIFIER, option.key(),
tableOptions.get(option)));
+ }
+ }
+ }
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ private static void configureDeserializationSchema(
+ ReadableConfig formatOptions,
InLongMsgTlogCsvDeserializationSchema.Builder schemaBuilder) {
+ formatOptions
+ .getOptional(TIME_FIELD_NAME)
+ .ifPresent(schemaBuilder::setTimeFieldName);
+
+ formatOptions
+ .getOptional(ATTRIBUTE_FIELD_NAME)
+ .ifPresent(schemaBuilder::setAttributesFieldName);
+
+ formatOptions
+ .getOptional(CHARSET)
+ .ifPresent(schemaBuilder::setCharset);
+
+ formatOptions
+ .getOptional(FIELD_DELIMITER)
+ .map(delimiter ->
StringEscapeUtils.unescapeJava(delimiter).charAt(0))
+ .ifPresent(schemaBuilder::setDelimiter);
+
+ formatOptions
+ .getOptional(QUOTE_CHARACTER)
+ .map(quote -> quote.charAt(0))
+ .ifPresent(schemaBuilder::setQuoteCharacter);
+
+ formatOptions
+ .getOptional(ESCAPE_CHARACTER)
+ .map(escape -> escape.charAt(0))
+ .ifPresent(schemaBuilder::setEscapeCharacter);
+
+
formatOptions.getOptional(NULL_LITERAL).ifPresent(schemaBuilder::setNullLiteral);
+
+
formatOptions.getOptional(IGNORE_ERRORS).ifPresent(schemaBuilder::setIgnoreErrors);
+ }
+}
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java
new file mode 100644
index 0000000000..2bf3ad6932
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java
@@ -0,0 +1,170 @@
+/*
+ * 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.inlong.sort.formats.inlongmsgtlogcsv;
+
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.formats.common.RowFormatInfo;
+import org.apache.inlong.sort.formats.inlongmsg.InLongMsgBody;
+import org.apache.inlong.sort.formats.inlongmsg.InLongMsgHead;
+
+import org.apache.flink.types.Row;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.Charset;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static
org.apache.inlong.sort.formats.base.TableFormatUtils.deserializeBasicField;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.INLONGMSG_ATTR_TIME_DT;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.INLONGMSG_ATTR_TIME_T;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.getPredefinedFields;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.parseAttr;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.parseDateTime;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.parseEpochTime;
+import static org.apache.inlong.sort.formats.util.StringUtils.splitCsv;
+
+/**
+ * Utilities for InLongMsgTlogCsv.
+ */
+public class InLongMsgTlogCsvUtils {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(InLongMsgTlogCsvUtils.class);
+
+ public static InLongMsgHead parseHead(String attr) {
+ Map<String, String> attributes = parseAttr(attr);
+
+ // Extracts time from the attributes
+ Timestamp time;
+
+ if (attributes.containsKey(INLONGMSG_ATTR_TIME_T)) {
+ String date = attributes.get(INLONGMSG_ATTR_TIME_T).trim();
+ time = parseDateTime(date);
+ } else if (attributes.containsKey(INLONGMSG_ATTR_TIME_DT)) {
+ String epoch = attributes.get(INLONGMSG_ATTR_TIME_DT).trim();
+ time = parseEpochTime(epoch);
+ } else {
+ throw new IllegalArgumentException(
+ "Could not find " + INLONGMSG_ATTR_TIME_T +
+ " or " + INLONGMSG_ATTR_TIME_DT + " in
attributes!");
+ }
+
+ // Extracts predefined fields from the attributes
+ List<String> predefinedFields = getPredefinedFields(attributes);
+
+ return new InLongMsgHead(attributes, null, time, predefinedFields);
+ }
+
+ public static InLongMsgBody parseBody(
+ byte[] bytes,
+ String charset,
+ char delimiter,
+ Character escapeChar,
+ Character quoteChar) {
+ String text;
+ if (bytes[0] == delimiter) {
+ text = new String(bytes, 1, bytes.length - 1,
Charset.forName(charset));
+ } else {
+ text = new String(bytes, Charset.forName(charset));
+ }
+
+ String[] segments = splitCsv(text, delimiter, escapeChar, quoteChar);
+
+ String tid = segments[0];
+ List<String> fields =
+ Arrays.stream(segments, 1,
segments.length).collect(Collectors.toList());
+
+ return new InLongMsgBody(bytes, tid, fields, Collections.emptyMap());
+ }
+
+ /**
+ * Deserializes the given fields into the row.
+ *
+ * @param rowFormatInfo The format information of the row.
+ * @param nullLiteral The literal for null values.
+ * @param predefinedFields The predefined fields.
+ * @param fields The fields.
+ * @return The row deserialized from the row.
+ */
+ public static Row deserializeRow(
+ RowFormatInfo rowFormatInfo,
+ String nullLiteral,
+ List<String> predefinedFields,
+ List<String> fields) {
+ String[] fieldNames = rowFormatInfo.getFieldNames();
+ FormatInfo[] fieldFormatInfos = rowFormatInfo.getFieldFormatInfos();
+
+ int actualNumFields = predefinedFields.size() + fields.size();
+ if (actualNumFields != fieldNames.length) {
+ LOG.warn("The number of fields mismatches: " + fieldNames.length +
+ " expected, but was " + actualNumFields + ".");
+ }
+
+ Row row = new Row(fieldNames.length);
+
+ for (int i = 0; i < predefinedFields.size(); ++i) {
+
+ if (i >= fieldNames.length) {
+ break;
+ }
+
+ String fieldName = fieldNames[i];
+ FormatInfo fieldFormatInfo = fieldFormatInfos[i];
+
+ String fieldText = predefinedFields.get(i);
+
+ Object field =
+ deserializeBasicField(
+ fieldName,
+ fieldFormatInfo,
+ fieldText,
+ nullLiteral);
+ row.setField(i, field);
+ }
+
+ for (int i = 0; i < fields.size(); ++i) {
+
+ if (i + predefinedFields.size() >= fieldNames.length) {
+ break;
+ }
+
+ String fieldName = fieldNames[i + predefinedFields.size()];
+ FormatInfo fieldFormatInfo = fieldFormatInfos[i +
predefinedFields.size()];
+
+ String fieldText = fields.get(i);
+
+ Object field =
+ deserializeBasicField(
+ fieldName,
+ fieldFormatInfo,
+ fieldText,
+ nullLiteral);
+ row.setField(i + predefinedFields.size(), field);
+ }
+
+ for (int i = predefinedFields.size() + fields.size(); i <
fieldNames.length; ++i) {
+ row.setField(i, null);
+ }
+
+ return row;
+ }
+}
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
new file mode 100644
index 0000000000..a5531edfdb
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/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.
+
+org.apache.inlong.sort.formats.inlongmsgtlogcsv.InLongMsgTlogCsvFormatFactory
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializerTest.java
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializerTest.java
new file mode 100644
index 0000000000..5929dadebe
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializerTest.java
@@ -0,0 +1,337 @@
+/*
+ * 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.inlong.sort.formats.inlongmsgtlogcsv;
+
+import org.apache.inlong.common.msg.InLongMsg;
+import org.apache.inlong.sort.formats.base.FieldToRowDataConverters;
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.formats.common.IntFormatInfo;
+import org.apache.inlong.sort.formats.common.RowFormatInfo;
+import org.apache.inlong.sort.formats.common.StringFormatInfo;
+import org.apache.inlong.sort.formats.inlongmsg.FailureHandler;
+import org.apache.inlong.sort.formats.inlongmsg.InLongMsgBody;
+import org.apache.inlong.sort.formats.inlongmsg.InLongMsgHead;
+
+import org.apache.flink.api.common.functions.util.ListCollector;
+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.data.TimestampData;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.util.Collector;
+import org.junit.Test;
+
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.table.api.DataTypes.MAP;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static
org.apache.inlong.sort.formats.base.TableFormatConstants.DEFAULT_CHARSET;
+import static
org.apache.inlong.sort.formats.base.TableFormatConstants.DEFAULT_DELIMITER;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgMetadata.ReadableMetadata.STREAMID;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.DEFAULT_ATTRIBUTES_FIELD_NAME;
+import static
org.apache.inlong.sort.formats.inlongmsg.InLongMsgUtils.DEFAULT_TIME_FIELD_NAME;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link InLongMsgTlogCsvFormatDeserializer}.
+ */
+public class InLongMsgTlogCsvFormatDeserializerTest {
+
+ private final FieldToRowDataConverters.FieldToRowDataConverter mapConvert =
+ FieldToRowDataConverters.createConverter(MAP(STRING(),
STRING()).getLogicalType());
+
+ private static final RowFormatInfo TEST_ROW_INFO =
+ new RowFormatInfo(
+ new String[]{"f1", "f2", "f3", "f4", "f5"},
+ new FormatInfo[]{
+ IntFormatInfo.INSTANCE,
+ IntFormatInfo.INSTANCE,
+ StringFormatInfo.INSTANCE,
+ StringFormatInfo.INSTANCE,
+ StringFormatInfo.INSTANCE
+ });
+
+ @Test
+ public void testExceptionHandler() throws Exception {
+ TestFailureHandler errorHandler = new TestFailureHandler();
+ InLongMsgTlogCsvFormatDeserializer deserializer =
+ new InLongMsgTlogCsvFormatDeserializer(
+ TEST_ROW_INFO,
+ DEFAULT_TIME_FIELD_NAME,
+ DEFAULT_ATTRIBUTES_FIELD_NAME,
+ DEFAULT_CHARSET,
+ DEFAULT_DELIMITER,
+ null,
+ null,
+ null,
+ Collections.emptyList(),
+ errorHandler);
+
+ InLongMsg inLongMsg1 = InLongMsg.newInLongMsg(true);
+ String attrs = "m=0&dt=1584806400000&__addcol1_=1&__addcol2_=test";
+ String body1 = "interfaceId1,field1,field2,field3";
+ String body2 = "interfaceId2,field1,field2,field3";
+ inLongMsg1.addMsg(attrs, body1.getBytes());
+ inLongMsg1.addMsg(attrs, body2.getBytes());
+
+ List<RowData> actualRows = new ArrayList<>();
+ Collector<RowData> collector = new ListCollector<>(actualRows);
+ deserializer.flatMap(inLongMsg1.buildArray(), collector);
+ assertEquals(2, errorHandler.getRowCount());
+
+ InLongMsg inLongMsg1Head = InLongMsg.newInLongMsg();
+ String abNormalAttrs = "m=0&__addcol1_=1&__addcol2_=test";
+ inLongMsg1Head.addMsg(abNormalAttrs, body1.getBytes());
+ inLongMsg1Head.addMsg(abNormalAttrs, body2.getBytes());
+ deserializer.flatMap(inLongMsg1Head.buildArray(), collector);
+ assertEquals(1, errorHandler.getHeadCount());
+ }
+
+ @Test
+ public void testNormal() throws Exception {
+ InLongMsg inLongMsg1 = InLongMsg.newInLongMsg(true);
+ String attrs = "m=0&dt=1584806400000&__addcol1_=1&__addcol2_=2";
+ String body1 = "interfaceId1,field1,field2,field3";
+ String body2 = "interfaceId2,field1,field2,field3";
+ inLongMsg1.addMsg(attrs, body1.getBytes());
+ inLongMsg1.addMsg(attrs, body2.getBytes());
+
+ Map<String, String> expectedAttributes = new HashMap<>();
+ expectedAttributes.put("m", "0");
+ expectedAttributes.put("dt", "1584806400000");
+ expectedAttributes.put("__addcol1_", "1");
+ expectedAttributes.put("__addcol2_", "2");
+
+ GenericRowData expectRowData1 = new GenericRowData(7);
+ expectRowData1.setField(0, TimestampData.fromTimestamp(new
Timestamp(1584806400000L)));
+ expectRowData1.setField(1, mapConvert.convert(expectedAttributes));
+ expectRowData1.setField(2, 1);
+ expectRowData1.setField(3, 2);
+ expectRowData1.setField(4, StringData.fromString("field1"));
+ expectRowData1.setField(5, StringData.fromString("field2"));
+ expectRowData1.setField(6, StringData.fromString("field3"));
+
+ GenericRowData expectRowData2 = new GenericRowData(7);
+ expectRowData2.setField(0, TimestampData.fromTimestamp(new
Timestamp(1584806400000L)));
+ expectRowData2.setField(1, mapConvert.convert(expectedAttributes));
+ expectRowData2.setField(2, 1);
+ expectRowData2.setField(3, 2);
+ expectRowData2.setField(4, StringData.fromString("field1"));
+ expectRowData2.setField(5, StringData.fromString("field2"));
+ expectRowData2.setField(6, StringData.fromString("field3"));
+
+ testRowDeserialization(
+ inLongMsg1.buildArray(),
+ Arrays.asList(expectRowData1, expectRowData2));
+ }
+
+ @Test
+ public void testDeleteHeadDelimiter() throws Exception {
+ InLongMsg inLongMsg1 = InLongMsg.newInLongMsg(true);
+ String attrs = "m=0&dt=1584806400000&__addcol1_=1&__addcol2_=2";
+ String body = ",interfaceId,field1,field2,field3";
+ inLongMsg1.addMsg(attrs, body.getBytes());
+
+ Map<String, String> expectedAttributes = new HashMap<>();
+ expectedAttributes.put("m", "0");
+ expectedAttributes.put("dt", "1584806400000");
+ expectedAttributes.put("__addcol1_", "1");
+ expectedAttributes.put("__addcol2_", "2");
+
+ GenericRowData expectRowData = new GenericRowData(7);
+ expectRowData.setField(0, TimestampData.fromTimestamp(new
Timestamp(1584806400000L)));
+ expectRowData.setField(1, mapConvert.convert(expectedAttributes));
+ expectRowData.setField(2, 1);
+ expectRowData.setField(3, 2);
+ expectRowData.setField(4, StringData.fromString("field1"));
+ expectRowData.setField(5, StringData.fromString("field2"));
+ expectRowData.setField(6, StringData.fromString("field3"));
+
+ testRowDeserialization(
+ inLongMsg1.buildArray(),
+ Collections.singletonList(expectRowData));
+ }
+
+ @Test
+ public void testUnmatchedFields1() throws Exception {
+ InLongMsg inLongMsg1 = InLongMsg.newInLongMsg(true);
+ String attrs = "m=0&dt=1584806400000&__addcol1_=1&__addcol2_=2";
+ String body1 = "interfaceId1,field1,field2";
+ String body2 = "interfaceId2,field1,field2,field3,field4";
+ inLongMsg1.addMsg(attrs, body1.getBytes());
+ inLongMsg1.addMsg(attrs, body2.getBytes());
+
+ Map<String, String> expectedAttributes = new HashMap<>();
+ expectedAttributes.put("m", "0");
+ expectedAttributes.put("dt", "1584806400000");
+ expectedAttributes.put("__addcol1_", "1");
+ expectedAttributes.put("__addcol2_", "2");
+
+ GenericRowData expectRowData1 = new GenericRowData(7);
+ expectRowData1.setField(0, TimestampData.fromTimestamp(new
Timestamp(1584806400000L)));
+ expectRowData1.setField(1, mapConvert.convert(expectedAttributes));
+ expectRowData1.setField(2, 1);
+ expectRowData1.setField(3, 2);
+ expectRowData1.setField(4, StringData.fromString("field1"));
+ expectRowData1.setField(5, StringData.fromString("field2"));
+ expectRowData1.setField(6, null);
+
+ GenericRowData expectRowData2 = new GenericRowData(7);
+ expectRowData2.setField(0, TimestampData.fromTimestamp(new
Timestamp(1584806400000L)));
+ expectRowData2.setField(1, mapConvert.convert(expectedAttributes));
+ expectRowData2.setField(2, 1);
+ expectRowData2.setField(3, 2);
+ expectRowData2.setField(4, StringData.fromString("field1"));
+ expectRowData2.setField(5, StringData.fromString("field2"));
+ expectRowData2.setField(6, StringData.fromString("field3"));
+
+ testRowDeserialization(
+ inLongMsg1.buildArray(),
+ Arrays.asList(expectRowData1, expectRowData2));
+ }
+
+ @Test
+ public void testUnmatchedFields2() throws Exception {
+ InLongMsg inLongMsg1 = InLongMsg.newInLongMsg(true);
+ String attrs = "m=0&dt=1584806400000&__addcol1_=1&__addcol2_=2&" +
+ "__addcol3_=3&__addcol4_=4&__addcol5_=5&__addcol6_=6";
+ String body = "interfaceId1,field1,field2,field3";
+ inLongMsg1.addMsg(attrs, body.getBytes());
+
+ Map<String, String> expectedAttributes = new HashMap<>();
+ expectedAttributes.put("m", "0");
+ expectedAttributes.put("dt", "1584806400000");
+ expectedAttributes.put("__addcol1_", "1");
+ expectedAttributes.put("__addcol2_", "2");
+ expectedAttributes.put("__addcol3_", "3");
+ expectedAttributes.put("__addcol4_", "4");
+ expectedAttributes.put("__addcol5_", "5");
+ expectedAttributes.put("__addcol6_", "6");
+
+ GenericRowData expectRowData = new GenericRowData(7);
+ expectRowData.setField(0, TimestampData.fromTimestamp(new
Timestamp(1584806400000L)));
+ expectRowData.setField(1, mapConvert.convert(expectedAttributes));
+ expectRowData.setField(2, 1);
+ expectRowData.setField(3, 2);
+ expectRowData.setField(4, StringData.fromString("3"));
+ expectRowData.setField(5, StringData.fromString("4"));
+ expectRowData.setField(6, StringData.fromString("5"));
+
+ testRowDeserialization(
+ inLongMsg1.buildArray(),
+ Collections.singletonList(expectRowData));
+ }
+
+ @Test
+ public void testRowType() {
+ InLongMsgTlogCsvFormatDeserializer deserializer =
+ new InLongMsgTlogCsvFormatDeserializer.Builder(TEST_ROW_INFO)
+ .setTimeFieldName("inlongmsg_time")
+ .setAttributesFieldName("inlongmsg_attributes")
+
.setMetadataKeys(Collections.singletonList(STREAMID.getKey()))
+ .build();
+
+ String[] expectedFieldNames = new String[]{
+ "inlongmsg_time",
+ "inlongmsg_attributes",
+ "f1",
+ "f2",
+ "f3",
+ "f4",
+ "f5",
+ "metadata-streamId"
+ };
+
+ LogicalType[] expectedFieldTypes = new LogicalType[]{
+ new TimestampType(),
+ new MapType(new VarCharType(), new VarCharType()),
+ new IntType(),
+ new IntType(),
+ new VarCharType(),
+ new VarCharType(),
+ new VarCharType(),
+ new VarCharType()
+ };
+ RowType expectedRowType = RowType.of(expectedFieldTypes,
expectedFieldNames);
+ assertEquals(InternalTypeInfo.of(expectedRowType),
deserializer.getProducedType());
+ }
+
+ private void testRowDeserialization(
+ byte[] bytes,
+ List<RowData> expectedRows) throws Exception {
+ InLongMsgTlogCsvFormatDeserializer deserializer =
+ new InLongMsgTlogCsvFormatDeserializer.Builder(TEST_ROW_INFO)
+ .setTimeFieldName("inlongmsg_time")
+ .setAttributesFieldName("inlongmsg_attributes")
+ .build();
+
+ List<RowData> actualRows = new ArrayList<>();
+ Collector<RowData> collector = new ListCollector<>(actualRows);
+
+ deserializer.flatMap(bytes, collector);
+
+ assertEquals(expectedRows, actualRows);
+ }
+
+ private static class TestFailureHandler implements FailureHandler {
+
+ private int headCount = 0;
+ private int bodyCount = 0;
+ private int rowCount = 0;
+
+ public int getHeadCount() {
+ return headCount;
+ }
+
+ public int getBodyCount() {
+ return bodyCount;
+ }
+
+ public int getRowCount() {
+ return rowCount;
+ }
+
+ @Override
+ public void onParsingHeadFailure(String attribute, Exception
exception) throws Exception {
+ headCount++;
+ }
+
+ @Override
+ public void onParsingBodyFailure(byte[] body, Exception exception)
throws Exception {
+ bodyCount++;
+ }
+
+ @Override
+ public void onConvertingRowFailure(InLongMsgHead head, InLongMsgBody
body, Exception exception)
+ throws Exception {
+ rowCount++;
+ }
+ }
+}
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatFactoryTest.java
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatFactoryTest.java
new file mode 100644
index 0000000000..33aed19033
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatFactoryTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.inlong.sort.formats.inlongmsgtlogcsv;
+
+import org.apache.inlong.sort.formats.common.DateFormatInfo;
+import org.apache.inlong.sort.formats.common.FormatInfo;
+import org.apache.inlong.sort.formats.common.FormatUtils;
+import org.apache.inlong.sort.formats.common.IntFormatInfo;
+import org.apache.inlong.sort.formats.common.MapFormatInfo;
+import org.apache.inlong.sort.formats.common.RowFormatInfo;
+import org.apache.inlong.sort.formats.common.StringFormatInfo;
+import org.apache.inlong.sort.formats.common.TimestampFormatInfo;
+
+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.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.TestDynamicTableFactory;
+import
org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static
org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link InLongMsgTlogCsvFormatFactory}.
+ */
+public class InLongMsgTlogCsvFormatFactoryTest {
+
+ public RowFormatInfo testFormatInfo;
+
+ public TypeInformation<RowData> testTypeInformation;
+
+ public ResolvedSchema resolvedSchema;
+
+ public DataType dataType;
+
+ @Before
+ public void setup() {
+ resolvedSchema =
+ ResolvedSchema.of(
+ Column.physical("time", DataTypes.TIMESTAMP()),
+ Column.physical("attributes",
DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING())),
+ Column.physical("student_name", DataTypes.STRING()),
+ Column.physical("score", DataTypes.INT()),
+ Column.physical("date", DataTypes.DATE()));
+ dataType = resolvedSchema.toPhysicalRowDataType();
+ RowType rowType = (RowType) dataType.getLogicalType();
+ testTypeInformation = InternalTypeInfo.of(rowType);
+ testFormatInfo = new RowFormatInfo(
+ new String[]{"time", "attributes", "student_name", "score",
"date"},
+ new FormatInfo[]{
+ new TimestampFormatInfo(),
+ new MapFormatInfo(StringFormatInfo.INSTANCE,
StringFormatInfo.INSTANCE),
+ StringFormatInfo.INSTANCE,
+ IntFormatInfo.INSTANCE,
+ new DateFormatInfo("yyyy-MM-dd")
+ });
+ }
+
+ @Test
+ public void testDeSchema() {
+ final InLongMsgTlogCsvDeserializationSchema expectedDeSer =
+ new InLongMsgTlogCsvDeserializationSchema.Builder(
+ testFormatInfo)
+ .setCharset("UTF-8")
+ .setDelimiter(';')
+ .setQuoteCharacter('\'')
+ .setEscapeCharacter('\\')
+ .setNullLiteral("n/a")
+ .build();
+ final Map<String, String> options = getAllOptions();
+ DeserializationSchema<RowData> actualDeser =
createDeserializationSchema(options);
+ assertEquals(expectedDeSer, actualDeser);
+ }
+
+ private DeserializationSchema<RowData> createDeserializationSchema(
+ Map<String, String> options) {
+ final DynamicTableSource actualSource =
createTableSource(resolvedSchema, options);
+ assert actualSource instanceof
TestDynamicTableFactory.DynamicTableSourceMock;
+ TestDynamicTableFactory.DynamicTableSourceMock sourceMock =
+ (TestDynamicTableFactory.DynamicTableSourceMock) actualSource;
+
+ return sourceMock.valueFormat.createRuntimeDecoder(
+ ScanRuntimeProviderContext.INSTANCE, dataType);
+ }
+
+ private Map<String, String> getAllOptions() {
+ final Map<String, String> options = new HashMap<>();
+ options.put("connector", TestDynamicTableFactory.IDENTIFIER);
+ options.put("target", "MyTarget");
+ options.put("buffer-size", "1000");
+
+ options.put("format", InLongMsgTlogCsvFormatFactory.IDENTIFIER);
+ options.put("inlong-msg-tlogcsv.row.format.info",
FormatUtils.marshall(testFormatInfo));
+ options.put("inlong-msg-tlogcsv.format.field-delimiter", ";");
+ options.put("inlong-msg-tlogcsv.format.quote-character", "'");
+ options.put("inlong-msg-tlogcsv.format.escape-character", "\\");
+ options.put("inlong-msg-tlogcsv.format.null-literal", "n/a");
+ return options;
+ }
+}
diff --git
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/resources/log4j-test.properties
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000000..e07d62aefe
--- /dev/null
+++
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-tlogcsv/src/test/resources/log4j-test.properties
@@ -0,0 +1,27 @@
+################################################################################
+# 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.
+################################################################################
+
+# Set root logger level to DEBUG and its only appender to A1.
+log4j.rootLogger=INFO, A1
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=[%t] %-5p %l %x - %m%n
diff --git a/inlong-sort/sort-formats/format-rowdata/pom.xml
b/inlong-sort/sort-formats/format-rowdata/pom.xml
index 84dae49f39..daf7b05ef5 100644
--- a/inlong-sort/sort-formats/format-rowdata/pom.xml
+++ b/inlong-sort/sort-formats/format-rowdata/pom.xml
@@ -41,6 +41,7 @@
<module>format-inlongmsg-rowdata-pb</module>
<module>format-inlongmsg-rowdata-kv</module>
<module>format-inlongmsg-rowdata-csv</module>
+ <module>format-inlongmsg-rowdata-tlogcsv</module>
</modules>
<properties>