This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new ff249bc  [FLINK-12979][formats] Allow empty line delimiter for 
CsvRowSerializationSchema
ff249bc is described below

commit ff249bce3dd4ef61518fb1e647e31e8f85640b61
Author: cyq89051127 <cha...@asiainfo.com>
AuthorDate: Sun Aug 25 15:01:43 2019 +0800

    [FLINK-12979][formats] Allow empty line delimiter for 
CsvRowSerializationSchema
    
    This closes #9529.
---
 docs/dev/table/connect.md                                     |  7 ++++---
 docs/dev/table/connect.zh.md                                  | 11 ++++++-----
 .../apache/flink/formats/csv/CsvRowSerializationSchema.java   |  4 ++--
 .../src/main/java/org/apache/flink/table/descriptors/Csv.java |  2 +-
 .../java/org/apache/flink/table/descriptors/CsvValidator.java |  2 +-
 .../flink/formats/csv/CsvRowDeSerializationSchemaTest.java    | 11 +++++++++++
 6 files changed, 25 insertions(+), 12 deletions(-)

diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md
index 975a0d0..5206620 100644
--- a/docs/dev/table/connect.md
+++ b/docs/dev/table/connect.md
@@ -1355,7 +1355,7 @@ The CSV format can be used as follows:
 
     .fieldDelimiter(';')         // optional: field delimiter character (',' 
by default)
     .lineDelimiter("\r\n")       // optional: line delimiter ("\n" by default;
-                                 //   otherwise "\r" or "\r\n" are allowed)
+                                 //   otherwise "\r", "\r\n", or "" are 
allowed)
     .quoteCharacter('\'')        // optional: quote character for enclosing 
field values ('"' by default)
     .allowComments()             // optional: ignores comment lines that start 
with '#' (disabled by default);
                                  //   if enabled, make sure to also ignore 
parse errors to allow empty rows
@@ -1383,7 +1383,7 @@ The CSV format can be used as follows:
 
     .field_delimiter(';')          # optional: field delimiter character (',' 
by default)
     .line_delimiter("\r\n")        # optional: line delimiter ("\n" by default;
-                                   #   otherwise "\r" or "\r\n" are allowed)
+                                   #   otherwise "\r", "\r\n", or "" are 
allowed)
     .quote_character('\'')         # optional: quote character for enclosing 
field values ('"' by default)
     .allow_comments()              # optional: ignores comment lines that 
start with '#' (disabled by default);
                                    #   if enabled, make sure to also ignore 
parse errors to allow empty rows
@@ -1410,7 +1410,8 @@ format:
   derive-schema: true
 
   field-delimiter: ";"         # optional: field delimiter character (',' by 
default)
-  line-delimiter: "\r\n"       # optional: line delimiter ("\n" by default; 
otherwise "\r" or "\r\n" are allowed)
+  line-delimiter: "\r\n"       # optional: line delimiter ("\n" by default;
+                               #   otherwise "\r", "\r\n", or "" are allowed)
   quote-character: "'"         # optional: quote character for enclosing field 
values ('"' by default)
   allow-comments: true         # optional: ignores comment lines that start 
with "#" (disabled by default);
                                #   if enabled, make sure to also ignore parse 
errors to allow empty rows
diff --git a/docs/dev/table/connect.zh.md b/docs/dev/table/connect.zh.md
index 6deb6af..9040cf1 100644
--- a/docs/dev/table/connect.zh.md
+++ b/docs/dev/table/connect.zh.md
@@ -1114,7 +1114,7 @@ The CSV format can be used as follows:
 
     .fieldDelimiter(';')         // optional: field delimiter character (',' 
by default)
     .lineDelimiter("\r\n")       // optional: line delimiter ("\n" by default;
-                                 //   otherwise "\r" or "\r\n" are allowed)
+                                 //   otherwise "\r", "\r\n", or "" are 
allowed)
     .quoteCharacter('\'')        // optional: quote character for enclosing 
field values ('"' by default)
     .allowComments()             // optional: ignores comment lines that start 
with '#' (disabled by default);
                                  //   if enabled, make sure to also ignore 
parse errors to allow empty rows
@@ -1142,9 +1142,9 @@ The CSV format can be used as follows:
 
     .field_delimiter(';')          # optional: field delimiter character (',' 
by default)
     .line_delimiter("\r\n")        # optional: line delimiter ("\n" by default;
-                                   #   otherwise "\r" or "\r\n" are allowed)
-    .quote_character('\'')         # optional: quote character for enclosing 
field values ('"' by default)
-    .allow_comments()              # optional: ignores comment lines that 
start with '#' (disabled by default);
+                                   #   otherwise "\r", "\r\n", or "" are 
allowed)
+    .quote_character("'")          # optional: quote character for enclosing 
field values ('"' by default)
+    .allow_comments()              # optional: ignores comment lines that 
start with "#" (disabled by default);
                                    #   if enabled, make sure to also ignore 
parse errors to allow empty rows
     .ignore_parse_errors()         # optional: skip fields and rows with parse 
errors instead of failing;
                                    #   fields are set to null in case of errors
@@ -1169,7 +1169,8 @@ format:
   derive-schema: true
 
   field-delimiter: ";"         # optional: field delimiter character (',' by 
default)
-  line-delimiter: "\r\n"       # optional: line delimiter ("\n" by default; 
otherwise "\r" or "\r\n" are allowed)
+  line-delimiter: "\r\n"       # optional: line delimiter ("\n" by default;
+                               #   otherwise "\r", "\r\n", or "" are allowed)
   quote-character: "'"         # optional: quote character for enclosing field 
values ('"' by default)
   allow-comments: true         # optional: ignores comment lines that start 
with "#" (disabled by default);
                                #   if enabled, make sure to also ignore parse 
errors to allow empty rows
diff --git 
a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java
 
b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java
index a65554f..b3c2de4 100644
--- 
a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java
+++ 
b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowSerializationSchema.java
@@ -116,9 +116,9 @@ public final class CsvRowSerializationSchema implements 
SerializationSchema<Row>
 
                public Builder setLineDelimiter(String delimiter) {
                        Preconditions.checkNotNull(delimiter, "Delimiter must 
not be null.");
-                       if (!delimiter.equals("\n") && !delimiter.equals("\r") 
&& !delimiter.equals("\r\n")) {
+                       if (!delimiter.equals("\n") && !delimiter.equals("\r") 
&& !delimiter.equals("\r\n") && !delimiter.equals("")) {
                                throw new IllegalArgumentException(
-                                       "Unsupported new line delimiter. Only 
\\n, \\r, or \\r\\n are supported.");
+                                       "Unsupported new line delimiter. Only 
\\n, \\r, \\r\\n, or empty string are supported.");
                        }
                        this.csvSchema = 
this.csvSchema.rebuild().setLineSeparator(delimiter).build();
                        return this;
diff --git 
a/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/Csv.java
 
b/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/Csv.java
index c467f1f..5fb3205 100644
--- 
a/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/Csv.java
+++ 
b/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/Csv.java
@@ -75,7 +75,7 @@ public class Csv extends FormatDescriptor {
        }
 
        /**
-        * Sets the line delimiter ("\n" by default; otherwise "\r" or "\r\n" 
are allowed).
+        * Sets the line delimiter ("\n" by default; otherwise "\r", "\r\n", or 
"" are allowed).
         *
         * @param delimiter the line delimiter
         */
diff --git 
a/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/CsvValidator.java
 
b/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/CsvValidator.java
index bc53971..2f7ebab 100644
--- 
a/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/CsvValidator.java
+++ 
b/flink-formats/flink-csv/src/main/java/org/apache/flink/table/descriptors/CsvValidator.java
@@ -44,7 +44,7 @@ public class CsvValidator extends FormatDescriptorValidator {
        public void validate(DescriptorProperties properties) {
                super.validate(properties);
                properties.validateString(FORMAT_FIELD_DELIMITER, true, 1, 1);
-               properties.validateEnumValues(FORMAT_LINE_DELIMITER, true, 
Arrays.asList("\r", "\n", "\r\n"));
+               properties.validateEnumValues(FORMAT_LINE_DELIMITER, true, 
Arrays.asList("\r", "\n", "\r\n", ""));
                properties.validateString(FORMAT_QUOTE_CHARACTER, true, 1, 1);
                properties.validateBoolean(FORMAT_ALLOW_COMMENTS, true);
                properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true);
diff --git 
a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvRowDeSerializationSchemaTest.java
 
b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvRowDeSerializationSchemaTest.java
index ece2e45..02b3e27 100644
--- 
a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvRowDeSerializationSchemaTest.java
+++ 
b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvRowDeSerializationSchemaTest.java
@@ -168,6 +168,17 @@ public class CsvRowDeSerializationSchemaTest {
                        serialize(serSchemaBuilder, Row.of("Test", 12, 
"Hello")));
        }
 
+       @Test
+       public void testEmptyLineDelimiter() throws Exception {
+               final TypeInformation<Row> rowInfo = Types.ROW(Types.STRING, 
Types.INT, Types.STRING);
+               final CsvRowSerializationSchema.Builder serSchemaBuilder = new 
CsvRowSerializationSchema.Builder(rowInfo)
+                               .setLineDelimiter("");
+
+               assertArrayEquals(
+                               "Test,12,Hello".getBytes(),
+                               serialize(serSchemaBuilder, Row.of("Test", 12, 
"Hello")));
+       }
+
        @Test(expected = IllegalArgumentException.class)
        public void testInvalidNesting() throws Exception {
                testNullableField(Types.ROW(Types.ROW(Types.STRING)), "FAIL", 
Row.of(Row.of("FAIL")));

Reply via email to