jihoonson commented on a change in pull request #10383:
URL: https://github.com/apache/druid/pull/10383#discussion_r503577153



##########
File path: 
core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -96,23 +101,45 @@ public void close() throws IOException
   public CloseableIterator<InputRowListPlusRawValues> sample() throws 
IOException
   {
     return intermediateRowIterator().map(row -> {
-      final Map<String, Object> rawColumns;
+
+      final List<Map<String, Object>> rawColumnsList;
       try {
-        rawColumns = toMap(row);
+        rawColumnsList = toMap(row);
       }
       catch (Exception e) {
-        return InputRowListPlusRawValues.of(null, new ParseException(e, 
"Unable to parse row [%s] into JSON", row));
+        return Collections.singletonList(InputRowListPlusRawValues.of(null,
+                                                                      new 
ParseException(e, "Unable to parse row [%s] into JSON", row)));
+      }
+
+      if (CollectionUtils.isNullOrEmpty(rawColumnsList)) {
+        return Collections.singletonList(InputRowListPlusRawValues.of(null,
+                                                                      new 
ParseException("No map object parsed for row [%s]", row)));
       }
+
+      List<InputRow> rows;
       try {
-        return InputRowListPlusRawValues.of(parseInputRows(row), rawColumns);
+        rows = parseInputRows(row);
       }
       catch (ParseException e) {
-        return InputRowListPlusRawValues.of(rawColumns, e);
+        return rawColumnsList.stream().map(rawColumn -> 
InputRowListPlusRawValues.of(rawColumn, e)).collect(Collectors.toList());
       }
       catch (IOException e) {
-        return InputRowListPlusRawValues.of(rawColumns, new ParseException(e, 
"Unable to parse row [%s] into inputRow", row));
+        ParseException exception = new ParseException(e, "Unable to parse row 
[%s] into inputRow", row);
+        return rawColumnsList.stream().map(rawColumn -> 
InputRowListPlusRawValues.of(rawColumn, 
exception)).collect(Collectors.toList());
+      }
+
+      if (rows.size() != rawColumnsList.size()) {
+        return Collections.singletonList(InputRowListPlusRawValues.of(null,

Review comment:
       I think we should set both `rows` and `rawColumnsList` in 
`InputRowListPlusRawValues` in this case, so that users will learn how they are 
different.

##########
File path: 
core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -96,23 +101,45 @@ public void close() throws IOException
   public CloseableIterator<InputRowListPlusRawValues> sample() throws 
IOException
   {
     return intermediateRowIterator().map(row -> {
-      final Map<String, Object> rawColumns;
+
+      final List<Map<String, Object>> rawColumnsList;
       try {
-        rawColumns = toMap(row);
+        rawColumnsList = toMap(row);
       }
       catch (Exception e) {
-        return InputRowListPlusRawValues.of(null, new ParseException(e, 
"Unable to parse row [%s] into JSON", row));
+        return Collections.singletonList(InputRowListPlusRawValues.of(null,
+                                                                      new 
ParseException(e, "Unable to parse row [%s] into JSON", row)));
+      }
+
+      if (CollectionUtils.isNullOrEmpty(rawColumnsList)) {
+        return Collections.singletonList(InputRowListPlusRawValues.of(null,
+                                                                      new 
ParseException("No map object parsed for row [%s]", row)));
       }
+
+      List<InputRow> rows;
       try {
-        return InputRowListPlusRawValues.of(parseInputRows(row), rawColumns);
+        rows = parseInputRows(row);
       }
       catch (ParseException e) {
-        return InputRowListPlusRawValues.of(rawColumns, e);
+        return rawColumnsList.stream().map(rawColumn -> 
InputRowListPlusRawValues.of(rawColumn, e)).collect(Collectors.toList());
       }
       catch (IOException e) {
-        return InputRowListPlusRawValues.of(rawColumns, new ParseException(e, 
"Unable to parse row [%s] into inputRow", row));
+        ParseException exception = new ParseException(e, "Unable to parse row 
[%s] into inputRow", row);
+        return rawColumnsList.stream().map(rawColumn -> 
InputRowListPlusRawValues.of(rawColumn, 
exception)).collect(Collectors.toList());
+      }

Review comment:
       This doesn't still seem correct to me.. The purpose of returning 
`rawColumn` and `exception` in `InputRowListPlusRawValues` is showing them in 
the sampler altogether, so that users will debug their data and input format 
easily. To do that, we should show exactly what the data was when a 
`ParseException` is thrown. Now, thinking about the new behavior parsing a list 
of JSONs into `InputRow`s, `row` will contain the list of JSONs here. If a 
`ParseException` was thrown while parsing one of them, we should return one 
`InputRowListPlusRawValues` which contains `row` (the whole list of JSONs) and 
the exception because we don't know exactly in which JSON in the list the 
`ParseException` was thrown from. For this, I still think we should change 
`rawValues` in `InputRowListPlusRawValues` to be `List<Map<String, Object>>`. I 
understand that you don't want to touch it as it is widely used in unit tests, 
but I'm not sure if we can fix this without touching it.




----------------------------------------------------------------
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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to