You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/01/06 07:17:07 UTC

[GitHub] [flink] wuchong commented on a change in pull request #14508: [FLINK-20773][format] Support allow-unescaped-control-chars option for JSON format.

wuchong commented on a change in pull request #14508:
URL: https://github.com/apache/flink/pull/14508#discussion_r552406084



##########
File path: docs/dev/table/connectors/formats/maxwell.zh.md
##########
@@ -187,6 +188,13 @@ Format Options
       <td>String</td>
       <td>Specify string literal to replace null key when <code>'maxwell-json.map-null-key.mode'</code> is LITERAL.</td>
     </tr>
+    <tr>
+      <td><h5>maxwell-json.allow-unescaped-control-chars</h5></td>
+      <td>选填</td>
+      <td style="word-wrap: break-word;">false</td>
+      <td>Boolean</td>
+      <td>是否允许数据中存在未转义的控制字符(值小于32的ASCII字符,包括制表符和换行符),若不允许,解析时会抛出异常`Illegal unquoted character...`</td>

Review comment:
       ```suggestion
         <td>是否允许数据中存在未转义的控制字符(值小于32的ASCII字符,包括制表符和换行符),若不允许,解析时会抛出异常<code>Illegal unquoted character...</code></td>
   ```
   We have to use `<code>` tags in table HTML, otherwise, it can be recognized. Please update others too.

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java
##########
@@ -68,12 +69,16 @@
     /** Timestamp format specification which is used to parse timestamp. */
     private final TimestampFormat timestampFormat;
 
+    /** Flag indicating whether to allow unescaped control chars in the data. */
+    private final boolean allowUnescapedControlChars;
+
     public JsonRowDataDeserializationSchema(
             RowType rowType,
             TypeInformation<RowData> resultTypeInfo,
             boolean failOnMissingField,
             boolean ignoreParseErrors,
-            TimestampFormat timestampFormat) {
+            TimestampFormat timestampFormat,
+            boolean allowUnescapedControlChars) {

Review comment:
       The parameters are super large. Could you refactor it into builder pattern? Just like `CsvRowDataDeserializationSchema`. This can reduce the effort to add a new parameter in the future. 

##########
File path: flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDataSerDeSchemaTest.java
##########
@@ -692,15 +727,27 @@ private void testParseErrors(TestSpec spec) throws Exception {
                                     "Failed to deserialize JSON '{\"map\":{\"key1\":\"123\", \"key2\":\"abc\"}}'."),
                     TestSpec.json("{\"id\":\"2019-11-12T18:00:12\"}")
                             .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0))))
+                            .expect(new Row(1))
                             .expectErrorMessage(
                                     "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12\"}'."),
                     TestSpec.json("{\"id\":\"2019-11-12T18:00:12+0800\"}")
                             .rowType(ROW(FIELD("id", TIMESTAMP_WITH_LOCAL_TIME_ZONE(0))))
+                            .expect(new Row(1))
                             .expectErrorMessage(
                                     "Failed to deserialize JSON '{\"id\":\"2019-11-12T18:00:12+0800\"}'."),
                     TestSpec.json("{\"id\":1,\"factor\":799.929496989092949698}")
                             .rowType(ROW(FIELD("id", INT()), FIELD("factor", DECIMAL(38, 18))))
-                            .expect(Row.of(1, new BigDecimal("799.929496989092949698"))));
+                            .expect(Row.of(1, new BigDecimal("799.929496989092949698"))),
+                    TestSpec.json("{\"id\":\"\tstring field\"}")
+                            .rowType(ROW(FIELD("id", STRING())))
+                            .expect(Row.of("	string field")),
+                    TestSpec.json("{\"id\":\"\tstring field\"}")
+                            .rowType(ROW(FIELD("id", STRING())))
+                            .expect(null) // it will return null when deserializing the byte[] to

Review comment:
       I think for the control chars configuration, we can add a separate test method. This can avoid to touch so many TestSpecs. The TestSpec is used to verify the general deserializing and serializing. 

##########
File path: docs/dev/table/connectors/formats/maxwell.md
##########
@@ -152,8 +152,9 @@ Format Options
       <td>optional</td>
       <td style="word-wrap: break-word;">false</td>
       <td>Boolean</td>
-      <td>Skip fields and rows with parse errors instead of failing.
-      Fields are set to null in case of errors.</td>
+      <td>Skip parse errors instead of failing. 
+      This message will be ignored and returned <code>null</code> when deserializing the <code>byte[]</code> to the <code>JsonNode</code> fails.(e.g the message with invalid json format, the message contains unescaped control characters) 
+      Otherwise fields are set to null in case of errors.</td>

Review comment:
       This is very confusing. Does it skip the message or return `null` or set field to `null`???

##########
File path: flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDataDeserializationSchema.java
##########
@@ -85,16 +90,22 @@ public JsonRowDataDeserializationSchema(
                 new JsonToRowDataConverters(failOnMissingField, ignoreParseErrors, timestampFormat)
                         .createConverter(checkNotNull(rowType));
         this.timestampFormat = timestampFormat;
+        this.allowUnescapedControlChars = allowUnescapedControlChars;
         boolean hasDecimalType =
                 LogicalTypeChecks.hasNested(rowType, t -> t instanceof DecimalType);
         if (hasDecimalType) {
             objectMapper.enable(DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS);
         }
+        objectMapper.configure(
+                JsonReadFeature.ALLOW_UNESCAPED_CONTROL_CHARS.mappedFeature(),
+                allowUnescapedControlChars);
     }
 
     @Override
     public RowData deserialize(byte[] message) throws IOException {
         try {
+            // it will return null if `ignoreParseErrors` is true
+            // and when executing `objectMapper.readTree(message)` throw an exception.

Review comment:
       This comment is hard to understand. Why exception will be thrown when `ignoreParseErrors` is enabled. 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org