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 d18232b54a [INLONG-10241][Sort] TLog format requires the ability to 
parse and determine if the first segment is present (#10243)
d18232b54a is described below

commit d18232b54ad8f74ddd83af986c6ecf52b595a225
Author: Mingyu Bao <[email protected]>
AuthorDate: Thu May 23 20:59:54 2024 +0800

    [INLONG-10241][Sort] TLog format requires the ability to parse and 
determine if the first segment is present (#10243)
---
 .../InLongMsgTlogCsvFormatDeserializer.java        | 24 +++++++++++++++++-----
 .../InLongMsgTlogCsvMixedFormatDeserializer.java   | 11 ++++++++--
 .../inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java    |  5 +++--
 .../InLongMsgTlogCsvFormatDeserializerTest.java    | 10 ++++++---
 .../sort/formats/inlongmsg/InLongMsgUtils.java     |  4 ++--
 .../InLongMsgCsvFormatDeserializerTest.java        |  4 ++++
 .../InLongMsgTlogCsvFormatDeserializer.java        | 23 ++++++++++++++++-----
 .../inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java    |  5 +++--
 .../InLongMsgTlogCsvFormatDeserializerTest.java    |  4 +++-
 9 files changed, 68 insertions(+), 22 deletions(-)

diff --git 
a/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializer.java
 
b/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializer.java
index a606ad14e8..e569d0f43f 100644
--- 
a/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializer.java
+++ 
b/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializer.java
@@ -97,6 +97,9 @@ public final class InLongMsgTlogCsvFormatDeserializer extends 
AbstractInLongMsgF
     @Nullable
     private final String nullLiteral;
 
+    @Nonnull
+    private Boolean isIncludeFirstSegment = false;
+
     public InLongMsgTlogCsvFormatDeserializer(
             @Nonnull RowFormatInfo rowFormatInfo,
             @Nullable String timeFieldName,
@@ -116,6 +119,7 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
                 escapeChar,
                 quoteChar,
                 nullLiteral,
+                false,
                 InLongMsgUtils.getDefaultExceptionHandler(ignoreErrors));
     }
 
@@ -128,6 +132,7 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
             @Nullable Character escapeChar,
             @Nullable Character quoteChar,
             @Nullable String nullLiteral,
+            @Nullable boolean isIncludeFirstSegment,
             @Nonnull FailureHandler failureHandler) {
         super(failureHandler);
 
@@ -139,6 +144,7 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
         this.escapeChar = escapeChar;
         this.quoteChar = quoteChar;
         this.nullLiteral = nullLiteral;
+        this.isIncludeFirstSegment = isIncludeFirstSegment;
     }
 
     @Override
@@ -154,7 +160,8 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
     @Override
     protected List<InLongMsgBody> parseBodyList(byte[] bytes) throws Exception 
{
         return Collections.singletonList(
-                InLongMsgTlogCsvUtils.parseBody(bytes, charset, delimiter, 
escapeChar, quoteChar));
+                InLongMsgTlogCsvUtils.parseBody(bytes, charset, delimiter, 
escapeChar,
+                        quoteChar, isIncludeFirstSegment));
     }
 
     @Override
@@ -183,11 +190,16 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
         private String timeFieldName = DEFAULT_TIME_FIELD_NAME;
         private String attributesFieldName = DEFAULT_ATTRIBUTES_FIELD_NAME;
         private Character delimiter = DEFAULT_DELIMITER;
-
+        private boolean isIncludeFirstSegment = false;
         public Builder(RowFormatInfo rowFormatInfo) {
             super(rowFormatInfo);
         }
 
+        public Builder setIsIncludeFirstSegment(boolean isIncludeFirstSegment) 
{
+            this.isIncludeFirstSegment = isIncludeFirstSegment;
+            return this;
+        }
+
         public Builder setTimeFieldName(String timeFieldName) {
             this.timeFieldName = timeFieldName;
             return this;
@@ -226,7 +238,8 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
                     escapeChar,
                     quoteChar,
                     nullLiteral,
-                    ignoreErrors);
+                    isIncludeFirstSegment,
+                    InLongMsgUtils.getDefaultExceptionHandler(ignoreErrors));
         }
     }
 
@@ -252,13 +265,14 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
                 delimiter.equals(that.delimiter) &&
                 Objects.equals(escapeChar, that.escapeChar) &&
                 Objects.equals(quoteChar, that.quoteChar) &&
-                Objects.equals(nullLiteral, that.nullLiteral);
+                Objects.equals(nullLiteral, that.nullLiteral) &&
+                Objects.equals(isIncludeFirstSegment, 
that.isIncludeFirstSegment);
     }
 
     @Override
     public int hashCode() {
         return Objects.hash(super.hashCode(), rowFormatInfo, timeFieldName,
                 attributesFieldName, charset, delimiter, escapeChar, quoteChar,
-                nullLiteral);
+                nullLiteral, isIncludeFirstSegment);
     }
 }
diff --git 
a/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvMixedFormatDeserializer.java
 
b/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvMixedFormatDeserializer.java
index 46723f058c..11f86d204e 100644
--- 
a/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvMixedFormatDeserializer.java
+++ 
b/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvMixedFormatDeserializer.java
@@ -69,13 +69,17 @@ public final class InLongMsgTlogCsvMixedFormatDeserializer
     @Nullable
     private final Character quoteChar;
 
+    @Nonnull
+    private Boolean isIncludeFirstSegment = false;
+
     public InLongMsgTlogCsvMixedFormatDeserializer(
             @Nonnull String charset,
             @Nonnull Character delimiter,
             @Nullable Character escapeChar,
             @Nullable Character quoteChar,
             @Nonnull Boolean ignoreErrors) {
-        this(charset, delimiter, escapeChar, quoteChar, 
InLongMsgUtils.getDefaultExceptionHandler(ignoreErrors));
+        this(charset, delimiter, escapeChar, quoteChar, false,
+                InLongMsgUtils.getDefaultExceptionHandler(ignoreErrors));
     }
 
     public InLongMsgTlogCsvMixedFormatDeserializer(
@@ -83,6 +87,7 @@ public final class InLongMsgTlogCsvMixedFormatDeserializer
             @Nonnull Character delimiter,
             @Nullable Character escapeChar,
             @Nullable Character quoteChar,
+            @Nonnull Boolean isIncludeFirstSegment,
             @Nonnull FailureHandler failureHandler) {
         super(failureHandler);
 
@@ -90,6 +95,7 @@ public final class InLongMsgTlogCsvMixedFormatDeserializer
         this.charset = charset;
         this.escapeChar = escapeChar;
         this.quoteChar = quoteChar;
+        this.isIncludeFirstSegment = isIncludeFirstSegment;
     }
 
     @Override
@@ -105,7 +111,8 @@ public final class InLongMsgTlogCsvMixedFormatDeserializer
     @Override
     protected List<InLongMsgBody> parseBodyList(byte[] bytes) throws Exception 
{
         return Collections.singletonList(
-                InLongMsgTlogCsvUtils.parseBody(bytes, charset, delimiter, 
escapeChar, quoteChar));
+                InLongMsgTlogCsvUtils.parseBody(bytes, charset, delimiter, 
escapeChar,
+                        quoteChar, isIncludeFirstSegment));
     }
 
     @Override
diff --git 
a/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java
 
b/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java
index 79784249e7..9e0e35952b 100644
--- 
a/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java
+++ 
b/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/main/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvUtils.java
@@ -79,7 +79,8 @@ public class InLongMsgTlogCsvUtils {
             String charset,
             char delimiter,
             Character escapeChar,
-            Character quoteChar) {
+            Character quoteChar,
+            boolean isIncludeFirstSegment) {
         String text;
         if (bytes[0] == delimiter) {
             text = new String(bytes, 1, bytes.length - 1, 
Charset.forName(charset));
@@ -91,7 +92,7 @@ public class InLongMsgTlogCsvUtils {
 
         String streamId = segments[0];
         List<String> fields =
-                Arrays.stream(segments, 1, 
segments.length).collect(Collectors.toList());
+                Arrays.stream(segments, (isIncludeFirstSegment ? 0 : 1), 
segments.length).collect(Collectors.toList());
 
         return new InLongMsgBody(bytes, streamId, fields, 
Collections.emptyMap());
     }
diff --git 
a/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializerTest.java
 
b/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializerTest.java
index f24e788787..19c18ec49c 100644
--- 
a/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializerTest.java
+++ 
b/inlong-sort/sort-formats/format-row/format-inlongmsg-tlogcsv/src/test/java/org/apache/inlong/sort/formats/inlongmsgtlogcsv/InLongMsgTlogCsvFormatDeserializerTest.java
@@ -52,7 +52,7 @@ public class InLongMsgTlogCsvFormatDeserializerTest {
 
     private static final RowFormatInfo TEST_ROW_INFO =
             new RowFormatInfo(
-                    new String[]{"f1", "f2", "f3", "f4", "f5"},
+                    new String[]{"__addcol1_", "__addcol2_", "f1", "f2", "f3"},
                     new FormatInfo[]{
                             IntFormatInfo.INSTANCE,
                             IntFormatInfo.INSTANCE,
@@ -74,6 +74,7 @@ public class InLongMsgTlogCsvFormatDeserializerTest {
                         null,
                         null,
                         null,
+                        false,
                         errorHandler);
 
         InLongMsg inLongMsg1 = InLongMsg.newInLongMsg(true);
@@ -193,10 +194,12 @@ public class InLongMsgTlogCsvFormatDeserializerTest {
                 "field1",
                 "field2",
                 "field3");
-
+        List<Row> expectedRows = new ArrayList<>();
+        expectedRows.add(expectedRow1);
+        expectedRows.add(expectedRow2);
         testRowDeserialization(
                 inLongMsg1.buildArray(),
-                Arrays.asList(expectedRow1, expectedRow2));
+                expectedRows);
     }
 
     @Test
@@ -238,6 +241,7 @@ public class InLongMsgTlogCsvFormatDeserializerTest {
                 new InLongMsgTlogCsvFormatDeserializer.Builder(TEST_ROW_INFO)
                         .setTimeFieldName("inlongmsg_time")
                         .setAttributesFieldName("inlongmsg_attributes")
+                        .setIsIncludeFirstSegment(false)
                         .build();
 
         List<Row> actualRows = new ArrayList<>();
diff --git 
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-base/src/main/java/org/apache/inlong/sort/formats/inlongmsg/InLongMsgUtils.java
 
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-base/src/main/java/org/apache/inlong/sort/formats/inlongmsg/InLongMsgUtils.java
index 02bca5450e..f48c1c7a0f 100644
--- 
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-base/src/main/java/org/apache/inlong/sort/formats/inlongmsg/InLongMsgUtils.java
+++ 
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-base/src/main/java/org/apache/inlong/sort/formats/inlongmsg/InLongMsgUtils.java
@@ -73,8 +73,8 @@ public class InLongMsgUtils {
     public static final String INLONGMSG_ATTR_TIME_DT = "dt";
     public static final String INLONGMSG_ATTR_ADD_COLUMN_PREFIX = "__addcol";
 
-    public static final String DEFAULT_TIME_FIELD_NAME = null;
-    public static final String DEFAULT_ATTRIBUTES_FIELD_NAME = null;
+    public static final String DEFAULT_TIME_FIELD_NAME = "inlongmsg_time";
+    public static final String DEFAULT_ATTRIBUTES_FIELD_NAME = 
"inlongmsg_attributes";
 
     private static final FieldToRowDataConverters.FieldToRowDataConverter 
TIME_FIELD_CONVERTER =
             FieldToRowDataConverters.createConverter(new TimestampType());
diff --git 
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-csv/src/test/java/org/apache/inlong/sort/formats/inlongmsgcsv/InLongMsgCsvFormatDeserializerTest.java
 
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-csv/src/test/java/org/apache/inlong/sort/formats/inlongmsgcsv/InLongMsgCsvFormatDeserializerTest.java
index 7c6159a0de..fb8e00ab41 100644
--- 
a/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-csv/src/test/java/org/apache/inlong/sort/formats/inlongmsgcsv/InLongMsgCsvFormatDeserializerTest.java
+++ 
b/inlong-sort/sort-formats/format-rowdata/format-inlongmsg-rowdata-csv/src/test/java/org/apache/inlong/sort/formats/inlongmsgcsv/InLongMsgCsvFormatDeserializerTest.java
@@ -140,6 +140,8 @@ public class InLongMsgCsvFormatDeserializerTest {
                         .build();
 
         String[] fieldNames = new String[]{
+                "inlongmsg_time",
+                "inlongmsg_attributes",
                 "f1",
                 "f2",
                 "f3",
@@ -149,6 +151,8 @@ public class InLongMsgCsvFormatDeserializerTest {
         };
 
         LogicalType[] fieldTypes = new LogicalType[]{
+                new TimestampType(),
+                new MapType(new VarCharType(), new VarCharType()),
                 new IntType(),
                 new IntType(),
                 new IntType(),
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
index c0cdc68b30..814466f079 100644
--- 
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
@@ -92,6 +92,8 @@ public final class InLongMsgTlogCsvFormatDeserializer extends 
AbstractInLongMsgF
     @Nullable
     private final Character quoteChar;
 
+    @Nonnull
+    private Boolean isIncludeFirstSegment = false;
     /**
      * The literal represented null values, default "".
      */
@@ -123,6 +125,7 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
                 quoteChar,
                 nullLiteral,
                 metadataKeys,
+                false,
                 InLongMsgUtils.getDefaultExceptionHandler(ignoreErrors));
     }
 
@@ -136,6 +139,7 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
             @Nullable Character quoteChar,
             @Nullable String nullLiteral,
             List<String> metadataKeys,
+            @Nonnull Boolean isIncludeFirstSegment,
             @Nonnull FailureHandler failureHandler) {
         super(failureHandler);
 
@@ -148,7 +152,7 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
         this.quoteChar = quoteChar;
         this.nullLiteral = nullLiteral;
         this.metadataKeys = metadataKeys;
-
+        this.isIncludeFirstSegment = isIncludeFirstSegment;
         converters = Arrays.stream(rowFormatInfo.getFieldFormatInfos())
                 .map(formatInfo -> FieldToRowDataConverters.createConverter(
                         TableFormatUtils.deriveLogicalType(formatInfo)))
@@ -172,7 +176,8 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
     @Override
     protected List<InLongMsgBody> parseBodyList(byte[] bytes) throws Exception 
{
         return Collections.singletonList(
-                InLongMsgTlogCsvUtils.parseBody(bytes, charset, delimiter, 
escapeChar, quoteChar));
+                InLongMsgTlogCsvUtils.parseBody(bytes, charset, delimiter, 
escapeChar,
+                        quoteChar, isIncludeFirstSegment));
     }
 
     @Override
@@ -204,6 +209,7 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
         private String attributesFieldName = DEFAULT_ATTRIBUTES_FIELD_NAME;
         private Character delimiter = DEFAULT_DELIMITER;
         private List<String> metadataKeys = Collections.emptyList();
+        private boolean isIncludeFirstSegment = false;
 
         public Builder(RowFormatInfo rowFormatInfo) {
             super(rowFormatInfo);
@@ -229,6 +235,11 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
             return this;
         }
 
+        public Builder setIncludeFirstSegment(boolean isIncludeFirstSegment) {
+            this.isIncludeFirstSegment = isIncludeFirstSegment;
+            return this;
+        }
+
         public InLongMsgTlogCsvFormatDeserializer build() {
             return new InLongMsgTlogCsvFormatDeserializer(
                     rowFormatInfo,
@@ -240,7 +251,8 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
                     quoteChar,
                     nullLiteral,
                     metadataKeys,
-                    ignoreErrors);
+                    isIncludeFirstSegment,
+                    InLongMsgUtils.getDefaultExceptionHandler(ignoreErrors));
         }
     }
 
@@ -267,13 +279,14 @@ public final class InLongMsgTlogCsvFormatDeserializer 
extends AbstractInLongMsgF
                 Objects.equals(escapeChar, that.escapeChar) &&
                 Objects.equals(quoteChar, that.quoteChar) &&
                 Objects.equals(nullLiteral, that.nullLiteral) &&
-                Objects.equals(metadataKeys, that.metadataKeys);
+                Objects.equals(metadataKeys, that.metadataKeys) &&
+                Objects.equals(isIncludeFirstSegment, 
that.isIncludeFirstSegment);
     }
 
     @Override
     public int hashCode() {
         return Objects.hash(super.hashCode(), rowFormatInfo, timeFieldName,
                 attributesFieldName, charset, delimiter, escapeChar, quoteChar,
-                nullLiteral, metadataKeys);
+                nullLiteral, metadataKeys, isIncludeFirstSegment);
     }
 }
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
index 8523ff544d..a216cf3429 100644
--- 
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
@@ -80,7 +80,8 @@ public class InLongMsgTlogCsvUtils {
             String charset,
             char delimiter,
             Character escapeChar,
-            Character quoteChar) {
+            Character quoteChar,
+            boolean isIncludeFirstSegment) {
         String text;
         if (bytes[0] == delimiter) {
             text = new String(bytes, 1, bytes.length - 1, 
Charset.forName(charset));
@@ -92,7 +93,7 @@ public class InLongMsgTlogCsvUtils {
 
         String tid = segments[0];
         List<String> fields =
-                Arrays.stream(segments, 1, 
segments.length).collect(Collectors.toList());
+                Arrays.stream(segments, (isIncludeFirstSegment ? 0 : 1), 
segments.length).collect(Collectors.toList());
 
         return new InLongMsgBody(bytes, tid, fields, Collections.emptyMap());
     }
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
index 8d44ae4dd6..7e83816b4d 100644
--- 
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
@@ -68,7 +68,7 @@ public class InLongMsgTlogCsvFormatDeserializerTest {
 
     private static final RowFormatInfo TEST_ROW_INFO =
             new RowFormatInfo(
-                    new String[]{"f1", "f2", "f3", "f4", "f5"},
+                    new String[]{"__addcol1_", "__addcol2_", "f1", "f2", "f3"},
                     new FormatInfo[]{
                             IntFormatInfo.INSTANCE,
                             IntFormatInfo.INSTANCE,
@@ -91,6 +91,7 @@ public class InLongMsgTlogCsvFormatDeserializerTest {
                         null,
                         null,
                         Collections.emptyList(),
+                        false,
                         errorHandler);
 
         InLongMsg inLongMsg1 = InLongMsg.newInLongMsg(true);
@@ -258,6 +259,7 @@ public class InLongMsgTlogCsvFormatDeserializerTest {
                 new InLongMsgTlogCsvFormatDeserializer.Builder(TEST_ROW_INFO)
                         .setTimeFieldName("inlongmsg_time")
                         .setAttributesFieldName("inlongmsg_attributes")
+                        .setIncludeFirstSegment(false)
                         
.setMetadataKeys(Collections.singletonList(STREAMID.getKey()))
                         .build();
 

Reply via email to