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



##########
File path: 
core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -96,23 +99,35 @@ 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)));
       }
+
+      List<InputRow> rows;
       try {
-        return InputRowListPlusRawValues.of(parseInputRows(row), rawColumns);
+        rows = parseInputRows(row);
       }
       catch (ParseException e) {
-        return InputRowListPlusRawValues.of(rawColumns, e);
+        return 
Collections.singletonList(InputRowListPlusRawValues.of(rawColumnsList.isEmpty() 
? null : rawColumnsList.get(0), e));

Review comment:
       > I checked the code again and find that there's no need to return raw 
column object when exception occurs. Because `InputSourceSample` re-throws 
exception once `InputRowListPlusRawValues.getParseException` returns exception.
   
   [Actually, there is a clause catching 
`ParseException`](https://github.com/apache/druid/blob/master/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java#L142-L146)
 in the sampler (this is a bad pattern by the way. We should refactor it at 
some point). In this case, the `rawColumns` is returned to show the raw data in 
the UI. I think the entire `rawColumnsList` should be returned to keep the 
current behaviour.

##########
File path: 
core/src/test/java/org/apache/druid/data/input/impl/JsonInputFormatTest.java
##########
@@ -69,7 +69,7 @@ public void testEquals()
               new ObjectMapper(),
               new ObjectMapper()
               )
-              .withIgnoredFields("objectMapper")
+              .withIgnoredFields("objectMapper", "lineSplittable")

Review comment:
       Hmm, good point. Yeah, it's not a good convention to include a mutable 
field in `equals()` or `hashCode()` because they can be used as a key of a 
`HashMap` or `HashSet`. I would like to suggest to make `lineSplittable` 
immutable, but add a new method `withLineSplittable(boolean lineSplittable)` 
instead of `setLineSplittable()` which creates a new `JsonInputFormat` instance 
with the given value.

##########
File path: 
core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -96,23 +99,35 @@ 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)));
       }
+
+      List<InputRow> rows;
       try {
-        return InputRowListPlusRawValues.of(parseInputRows(row), rawColumns);
+        rows = parseInputRows(row);
       }
       catch (ParseException e) {
-        return InputRowListPlusRawValues.of(rawColumns, e);
+        return 
Collections.singletonList(InputRowListPlusRawValues.of(rawColumnsList.isEmpty() 
? null : rawColumnsList.get(0), e));

Review comment:
       There are some test failing in `InputSourceSamplerTest`. It seems like 
because of this change.

##########
File path: 
core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -96,23 +99,35 @@ 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)));
       }
+
+      List<InputRow> rows;
       try {
-        return InputRowListPlusRawValues.of(parseInputRows(row), rawColumns);
+        rows = parseInputRows(row);
       }
       catch (ParseException e) {
-        return InputRowListPlusRawValues.of(rawColumns, e);
+        return 
Collections.singletonList(InputRowListPlusRawValues.of(rawColumnsList.isEmpty() 
? null : rawColumnsList.get(0), e));

Review comment:
       There are some tests failing in `InputSourceSamplerTest`. It seems like 
because of this change.




----------------------------------------------------------------
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