You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/02/14 11:40:46 UTC

[GitHub] [druid] LakshSingla opened a new pull request #12259: Make ParseExceptions more informative

LakshSingla opened a new pull request #12259:
URL: https://github.com/apache/druid/pull/12259


   ### Description
   
   This PR aims to make the `ParseExceptions` in Druid more informative, by adding the input line number where the exception occurred, and the source which generated the exception. A `ParseException.Builder` is also added to make the creation of the `ParseException`s easier.
   
   This will also help in triaging the issues when `InputSourceReader` generates `ParseException`s because it can point to the specific `InputEntity` which caused the exception (while trying to read it). 
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `ParseException`
    * `InputEntityReader`'s subclasses
   
   <hr>
   
   This PR has:
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   


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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 merged pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 merged pull request #12259:
URL: https://github.com/apache/druid/pull/12259


   


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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r811445052



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,51 +111,126 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {
 
-      final List<Map<String, Object>> rawColumnsList;
-      try {
-        rawColumnsList = toMap(row);
-      }
-      catch (Exception e) {
-        return InputRowListPlusRawValues.of(null,
-                                            new ParseException(String.valueOf(row), e, "Unable to parse row [%s] into JSON", row));
-      }
+    final CloseableIteratorWithMetadata<T> delegate = intermediateRowIteratorWithMetadata();
+    final BiFunction<T, Map<String, Object>, InputRowListPlusRawValues> samplingFunction =
+        (row, metadata) -> {
 
-      if (CollectionUtils.isNullOrEmpty(rawColumnsList)) {
-        return InputRowListPlusRawValues.of(null,
-                                            new ParseException(String.valueOf(row), "No map object parsed for row [%s]", row));
-      }
+          final List<Map<String, Object>> rawColumnsList;
+          try {
+            rawColumnsList = toMap(row);
+          }
+          catch (Exception e) {
+            return InputRowListPlusRawValues.of(
+                null,
+                new ParseException(String.valueOf(row), e, buildParseExceptionMessage(
+                    StringUtils.nonStrictFormat("Unable to parse row [%s] into JSON", row),
+                    source(),
+                    null,
+                    metadata
+                ))
+            );
+          }
 
-      List<InputRow> rows;
-      try {
-        rows = parseInputRows(row);
-      }
-      catch (ParseException e) {
-        return InputRowListPlusRawValues.ofList(rawColumnsList, e);
+          if (CollectionUtils.isNullOrEmpty(rawColumnsList)) {
+            return InputRowListPlusRawValues.of(
+                null,
+                new ParseException(String.valueOf(row), buildParseExceptionMessage(
+                    StringUtils.nonStrictFormat("No map object parsed for row [%s]", row),
+                    source(),
+                    null,
+                    metadata
+                ))
+            );
+          }
+
+          List<InputRow> rows;
+          try {
+            rows = parseInputRows(row);
+          }
+          catch (ParseException e) {
+            return InputRowListPlusRawValues.ofList(rawColumnsList, new ParseException(
+                String.valueOf(row),
+                e,
+                buildParseExceptionMessage(e.getMessage(), source(), null, metadata)
+            ));
+          }
+          catch (IOException e) {
+            ParseException exception = new ParseException(String.valueOf(row), e, buildParseExceptionMessage(
+                StringUtils.nonStrictFormat("Unable to parse row [%s] into inputRow", row),
+                source(),
+                null,
+                metadata
+            ));
+            return InputRowListPlusRawValues.ofList(rawColumnsList, exception);
+          }
+
+          return InputRowListPlusRawValues.ofList(rawColumnsList, rows);
+        };
+
+    return new CloseableIterator<InputRowListPlusRawValues>()
+    {
+      @Override
+      public void close() throws IOException
+      {
+        delegate.close();
       }
-      catch (IOException e) {
-        ParseException exception = new ParseException(String.valueOf(row), e, "Unable to parse row [%s] into inputRow", row);
-        return InputRowListPlusRawValues.ofList(rawColumnsList, exception);
+
+      @Override
+      public boolean hasNext()
+      {
+        return delegate.hasNext();
       }
 
-      return InputRowListPlusRawValues.ofList(rawColumnsList, rows);
-    });
+      @Override
+      public InputRowListPlusRawValues next()
+      {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+
+        return samplingFunction.apply(delegate.next(), delegate.metadata());
+      }
+    };
   }
 
   /**
    * Creates an iterator of intermediate rows. The returned rows will be consumed by {@link #parseInputRows} and
-   * {@link #toMap}.
+   * {@link #toMap}. Either this or {@link #intermediateRowIteratorWithMetadata()} should be implemented
+   */
+  protected CloseableIterator<T> intermediateRowIterator() throws IOException
+  {
+    throw new UnsupportedEncodingException("intermediateRowIterator not implemented");

Review comment:
       Added accidentally




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r815679114



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -155,6 +259,39 @@ public void close() throws IOException
    */
   protected abstract List<Map<String, Object>> toMap(T intermediateRow) throws IOException;
 
+  /**
+   * A helper method which enriches the base parse exception message with additional information. The returned message
+   * has a format: "baseExceptionMessage (key1: value1, key2: value2)" if additional properties are present. Else it
+   * returns the baseException message without any modification
+   */
+  private static String buildParseExceptionMessage(
+      @Nonnull String baseExceptionMessage,
+      @Nullable InputEntity source,
+      @Nullable Long recordNumber,
+      @Nullable Map<String, Object> metadata
+  )
+  {
+    StringBuilder sb = new StringBuilder();
+    if (source != null && source.getUri() != null) {
+      sb.append(StringUtils.format("Path: %s, ", source.getUri()));
+    }
+    if (recordNumber != null) {
+      sb.append(StringUtils.format("Record: %d, ", recordNumber));
+    }
+    if (metadata != null) {
+      metadata.entrySet().stream()
+              .map(entry -> StringUtils.format("%s: %s, ", entry.getKey(), entry.getValue().toString()))
+              .forEach(sb::append);
+    }
+    if (sb.length() == 0) {
+      return baseExceptionMessage;
+    }
+    sb.setLength(sb.length() - 2); // Erase the last stray ", "
+    sb.insert(0, " ("); // Wrap the additional information in brackets

Review comment:
       this can be costly. Instead, you can just return baseExceptionMessage + "(" +  sb + ")" 




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r810817086



##########
File path: core/src/main/java/org/apache/druid/data/input/TextReader.java
##########
@@ -65,8 +68,17 @@ public InputRowSchema getInputRowSchema()
       processHeaderLine(delegate.nextLine());
     }
 
-    return new CloseableIterator<String>()
+    return new CloseableIteratorWithMetadata<String>()
     {
+      private final long currentLineNumber = numHeaderLines + (needsToProcessHeaderLine() ? 1 : 0);
+      final Map<String, Object> metadata = new HashMap<>(ImmutableMap.of("lineNumber", currentLineNumber));

Review comment:
       "lineNumber" can be just "line" and should be declared as a constant. 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -155,6 +248,32 @@ public void close() throws IOException
    */
   protected abstract List<Map<String, Object>> toMap(T intermediateRow) throws IOException;
 
+  private String buildParseExceptionMessage(
+      String baseExceptionMessage,
+      @Nullable InputEntity source,
+      @Nullable Long recordNumber,
+      @Nullable Map<String, Object> metadata
+  )
+  {
+    StringBuilder sb = new StringBuilder(baseExceptionMessage);
+    List<String> temp = new ArrayList<>();
+    if (source != null && source.getUri() != null) {
+      temp.add(StringUtils.format("Source info:[%s]", source.getUri()));
+    }
+    if (recordNumber != null) {
+      temp.add(StringUtils.format("Record number:[%d]", recordNumber));
+    }
+    if (metadata != null && !metadata.isEmpty()) {
+      temp.add(StringUtils.format("Additional info:%s", metadata));

Review comment:
       this is how I think it should be formatted
   "Source info: [Path: source.getUri(), Record number : recordNumber, key1: value1, key2 : value2]"
   
   where key1 and key2 are metadata entries. 

##########
File path: core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIteratorWithMetadata.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.common.parsers;
+
+import org.apache.druid.data.input.IntermediateRowParsingReader;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+/**
+ * Like {@link CloseableIterator}, but has a metadata() method, which returns "metadata", which is effectively a Map<String, Object>
+ * about the last value returned by next()
+ *
+ * The returned metadata is read-only and cannot be modified.
+ *
+ * This metadata can be used as additional information to pin-point the root cause of a parse exception.
+ * So it can include information that helps with such exercise. For example, for a {@link org.apache.druid.data.input.TextReader}
+ * that information can be the line number. Only per row context needs to be passed here so for kafka it could be an offset.
+ * The source information is already available via {@link IntermediateRowParsingReader#source()} method and needn't be included
+ */
+public interface CloseableIteratorWithMetadata<T> extends CloseableIterator<T>
+{
+  Map<String, Object> metadata();
+
+  static <T> CloseableIteratorWithMetadata<T> fromCloseableIterator(CloseableIterator<T> delegate)

Review comment:
       Please add some description here. 

##########
File path: core/src/main/java/org/apache/druid/data/input/TextReader.java
##########
@@ -65,8 +68,17 @@ public InputRowSchema getInputRowSchema()
       processHeaderLine(delegate.nextLine());
     }
 
-    return new CloseableIterator<String>()
+    return new CloseableIteratorWithMetadata<String>()
     {
+      private final long currentLineNumber = numHeaderLines + (needsToProcessHeaderLine() ? 1 : 0);
+      final Map<String, Object> metadata = new HashMap<>(ImmutableMap.of("lineNumber", currentLineNumber));

Review comment:
       instead of keeping a map around, you can just keep the line number and return it inside a map whenever `metadata()` is called. 

##########
File path: core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIteratorWithMetadata.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.common.parsers;
+
+import org.apache.druid.data.input.IntermediateRowParsingReader;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+/**
+ * Like {@link CloseableIterator}, but has a metadata() method, which returns "metadata", which is effectively a Map<String, Object>
+ * about the last value returned by next()
+ *
+ * The returned metadata is read-only and cannot be modified.
+ *
+ * This metadata can be used as additional information to pin-point the root cause of a parse exception.
+ * So it can include information that helps with such exercise. For example, for a {@link org.apache.druid.data.input.TextReader}
+ * that information can be the line number. Only per row context needs to be passed here so for kafka it could be an offset.
+ * The source information is already available via {@link IntermediateRowParsingReader#source()} method and needn't be included
+ */
+public interface CloseableIteratorWithMetadata<T> extends CloseableIterator<T>
+{
+  Map<String, Object> metadata();

Review comment:
       This method should be called currentMetadata()

##########
File path: core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIteratorWithMetadata.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.common.parsers;
+
+import org.apache.druid.data.input.IntermediateRowParsingReader;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+/**
+ * Like {@link CloseableIterator}, but has a metadata() method, which returns "metadata", which is effectively a Map<String, Object>
+ * about the last value returned by next()
+ *
+ * The returned metadata is read-only and cannot be modified.
+ *
+ * This metadata can be used as additional information to pin-point the root cause of a parse exception.
+ * So it can include information that helps with such exercise. For example, for a {@link org.apache.druid.data.input.TextReader}
+ * that information can be the line number. Only per row context needs to be passed here so for kafka it could be an offset.
+ * The source information is already available via {@link IntermediateRowParsingReader#source()} method and needn't be included
+ */
+public interface CloseableIteratorWithMetadata<T> extends CloseableIterator<T>
+{
+  Map<String, Object> metadata();
+
+  static <T> CloseableIteratorWithMetadata<T> fromCloseableIterator(CloseableIterator<T> delegate)

Review comment:
       this is similar to CloseableIterator.withEmptyBaggage.

##########
File path: core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIteratorWithMetadata.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.common.parsers;
+
+import org.apache.druid.data.input.IntermediateRowParsingReader;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+/**
+ * Like {@link CloseableIterator}, but has a metadata() method, which returns "metadata", which is effectively a Map<String, Object>
+ * about the last value returned by next()
+ *
+ * The returned metadata is read-only and cannot be modified.
+ *
+ * This metadata can be used as additional information to pin-point the root cause of a parse exception.
+ * So it can include information that helps with such exercise. For example, for a {@link org.apache.druid.data.input.TextReader}
+ * that information can be the line number. Only per row context needs to be passed here so for kafka it could be an offset.
+ * The source information is already available via {@link IntermediateRowParsingReader#source()} method and needn't be included
+ */
+public interface CloseableIteratorWithMetadata<T> extends CloseableIterator<T>
+{
+  Map<String, Object> metadata();
+
+  static <T> CloseableIteratorWithMetadata<T> fromCloseableIterator(CloseableIterator<T> delegate)

Review comment:
       ```suggestion
     static <T> CloseableIteratorWithMetadata<T> withEmptyMetadata(CloseableIterator<T> delegate)
   ```

##########
File path: core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIteratorWithMetadata.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.common.parsers;
+
+import org.apache.druid.data.input.IntermediateRowParsingReader;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+/**
+ * Like {@link CloseableIterator}, but has a metadata() method, which returns "metadata", which is effectively a Map<String, Object>
+ * about the last value returned by next()
+ *
+ * The returned metadata is read-only and cannot be modified.
+ *
+ * This metadata can be used as additional information to pin-point the root cause of a parse exception.
+ * So it can include information that helps with such exercise. For example, for a {@link org.apache.druid.data.input.TextReader}
+ * that information can be the line number. Only per row context needs to be passed here so for kafka it could be an offset.
+ * The source information is already available via {@link IntermediateRowParsingReader#source()} method and needn't be included
+ */
+public interface CloseableIteratorWithMetadata<T> extends CloseableIterator<T>
+{
+  Map<String, Object> metadata();

Review comment:
       Please add javadocs for method too and what should implementor put in this result map? 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,51 +111,126 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {
 
-      final List<Map<String, Object>> rawColumnsList;
-      try {
-        rawColumnsList = toMap(row);
-      }
-      catch (Exception e) {
-        return InputRowListPlusRawValues.of(null,
-                                            new ParseException(String.valueOf(row), e, "Unable to parse row [%s] into JSON", row));
-      }
+    final CloseableIteratorWithMetadata<T> delegate = intermediateRowIteratorWithMetadata();
+    final BiFunction<T, Map<String, Object>, InputRowListPlusRawValues> samplingFunction =
+        (row, metadata) -> {
 
-      if (CollectionUtils.isNullOrEmpty(rawColumnsList)) {
-        return InputRowListPlusRawValues.of(null,
-                                            new ParseException(String.valueOf(row), "No map object parsed for row [%s]", row));
-      }
+          final List<Map<String, Object>> rawColumnsList;
+          try {
+            rawColumnsList = toMap(row);
+          }
+          catch (Exception e) {
+            return InputRowListPlusRawValues.of(
+                null,
+                new ParseException(String.valueOf(row), e, buildParseExceptionMessage(
+                    StringUtils.nonStrictFormat("Unable to parse row [%s] into JSON", row),
+                    source(),
+                    null,
+                    metadata
+                ))
+            );
+          }
 
-      List<InputRow> rows;
-      try {
-        rows = parseInputRows(row);
-      }
-      catch (ParseException e) {
-        return InputRowListPlusRawValues.ofList(rawColumnsList, e);
+          if (CollectionUtils.isNullOrEmpty(rawColumnsList)) {
+            return InputRowListPlusRawValues.of(
+                null,
+                new ParseException(String.valueOf(row), buildParseExceptionMessage(
+                    StringUtils.nonStrictFormat("No map object parsed for row [%s]", row),
+                    source(),
+                    null,
+                    metadata
+                ))
+            );
+          }
+
+          List<InputRow> rows;
+          try {
+            rows = parseInputRows(row);
+          }
+          catch (ParseException e) {
+            return InputRowListPlusRawValues.ofList(rawColumnsList, new ParseException(
+                String.valueOf(row),
+                e,
+                buildParseExceptionMessage(e.getMessage(), source(), null, metadata)
+            ));
+          }
+          catch (IOException e) {
+            ParseException exception = new ParseException(String.valueOf(row), e, buildParseExceptionMessage(
+                StringUtils.nonStrictFormat("Unable to parse row [%s] into inputRow", row),
+                source(),
+                null,
+                metadata
+            ));
+            return InputRowListPlusRawValues.ofList(rawColumnsList, exception);
+          }
+
+          return InputRowListPlusRawValues.ofList(rawColumnsList, rows);
+        };
+
+    return new CloseableIterator<InputRowListPlusRawValues>()
+    {
+      @Override
+      public void close() throws IOException
+      {
+        delegate.close();
       }
-      catch (IOException e) {
-        ParseException exception = new ParseException(String.valueOf(row), e, "Unable to parse row [%s] into inputRow", row);
-        return InputRowListPlusRawValues.ofList(rawColumnsList, exception);
+
+      @Override
+      public boolean hasNext()
+      {
+        return delegate.hasNext();
       }
 
-      return InputRowListPlusRawValues.ofList(rawColumnsList, rows);
-    });
+      @Override
+      public InputRowListPlusRawValues next()
+      {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+
+        return samplingFunction.apply(delegate.next(), delegate.metadata());
+      }
+    };
   }
 
   /**
    * Creates an iterator of intermediate rows. The returned rows will be consumed by {@link #parseInputRows} and
-   * {@link #toMap}.
+   * {@link #toMap}. Either this or {@link #intermediateRowIteratorWithMetadata()} should be implemented
+   */
+  protected CloseableIterator<T> intermediateRowIterator() throws IOException
+  {
+    throw new UnsupportedEncodingException("intermediateRowIterator not implemented");

Review comment:
       UnsupportedEncodingException doesn't look right. It should be UnsupportedOperationException here. 

##########
File path: core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIteratorWithMetadata.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.common.parsers;
+
+import org.apache.druid.data.input.IntermediateRowParsingReader;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+/**
+ * Like {@link CloseableIterator}, but has a metadata() method, which returns "metadata", which is effectively a Map<String, Object>
+ * about the last value returned by next()

Review comment:
       ```suggestion
    * about the source of last value returned by next()
   ```

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,51 +111,126 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {
 
-      final List<Map<String, Object>> rawColumnsList;
-      try {
-        rawColumnsList = toMap(row);
-      }
-      catch (Exception e) {
-        return InputRowListPlusRawValues.of(null,
-                                            new ParseException(String.valueOf(row), e, "Unable to parse row [%s] into JSON", row));
-      }
+    final CloseableIteratorWithMetadata<T> delegate = intermediateRowIteratorWithMetadata();
+    final BiFunction<T, Map<String, Object>, InputRowListPlusRawValues> samplingFunction =
+        (row, metadata) -> {

Review comment:
       Instead of this big lambda, can you create a static method and pass the name of the method here in lambda expression here. 

##########
File path: core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIteratorWithMetadata.java
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.java.util.common.parsers;
+
+import org.apache.druid.data.input.IntermediateRowParsingReader;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+/**
+ * Like {@link CloseableIterator}, but has a metadata() method, which returns "metadata", which is effectively a Map<String, Object>
+ * about the last value returned by next()
+ *
+ * The returned metadata is read-only and cannot be modified.
+ *
+ * This metadata can be used as additional information to pin-point the root cause of a parse exception.
+ * So it can include information that helps with such exercise. For example, for a {@link org.apache.druid.data.input.TextReader}
+ * that information can be the line number. Only per row context needs to be passed here so for kafka it could be an offset.
+ * The source information is already available via {@link IntermediateRowParsingReader#source()} method and needn't be included
+ */
+public interface CloseableIteratorWithMetadata<T> extends CloseableIterator<T>
+{
+  Map<String, Object> metadata();
+
+  static <T> CloseableIteratorWithMetadata<T> fromCloseableIterator(CloseableIterator<T> delegate)
+  {
+    return new CloseableIteratorWithMetadata<T>()
+    {
+
+      @Override
+      public Map<String, Object> metadata()
+      {
+        return Collections.emptyMap();
+      }
+
+      @Override
+      public void close() throws IOException
+      {
+        delegate.close();
+      }
+
+      @Override
+      public boolean hasNext()
+      {
+        return delegate.hasNext();
+      }
+
+      @Override
+      public T next()
+      {
+        if (!hasNext()) {

Review comment:
       you can directly call delegate.next() here. 




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r807563063



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -155,6 +239,24 @@ public void close() throws IOException
    */
   protected abstract List<Map<String, Object>> toMap(T intermediateRow) throws IOException;
 
+  private String buildParseExceptionMessage(
+      String formatString,
+      @Nullable InputEntity source,
+      @Nullable Long recordNumber,
+      Map<String, Object> metadata,
+      Object... baseArgs
+  )
+  {
+    Map<String, Object> temp = Maps.newHashMap(metadata);
+    if (source != null) {
+      temp.put("source", source.getUri());
+    }
+    if (recordNumber != null) {
+      temp.put("recordNumber", recordNumber);
+    }
+    return StringUtils.nonStrictFormat(formatString, baseArgs, temp);

Review comment:
       Separated additional information from the base exception message. 




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r811429743



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,51 +111,126 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {
 
-      final List<Map<String, Object>> rawColumnsList;
-      try {
-        rawColumnsList = toMap(row);
-      }
-      catch (Exception e) {
-        return InputRowListPlusRawValues.of(null,
-                                            new ParseException(String.valueOf(row), e, "Unable to parse row [%s] into JSON", row));
-      }
+    final CloseableIteratorWithMetadata<T> delegate = intermediateRowIteratorWithMetadata();
+    final BiFunction<T, Map<String, Object>, InputRowListPlusRawValues> samplingFunction =
+        (row, metadata) -> {

Review comment:
       The function uses `toMap` which is overridden in the subclasses, so creating a new non-static method




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r807563756



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,39 +116,79 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {

Review comment:
       Removed the one-off implementation, and pushed it to the `IntermediateRowParsingReader` itself. 




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r806836691



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -155,6 +239,24 @@ public void close() throws IOException
    */
   protected abstract List<Map<String, Object>> toMap(T intermediateRow) throws IOException;
 
+  private String buildParseExceptionMessage(
+      String formatString,
+      @Nullable InputEntity source,
+      @Nullable Long recordNumber,
+      Map<String, Object> metadata,

Review comment:
       this can be null too I suppose. since not every implementation has metadata. 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -135,9 +198,30 @@ public void close() throws IOException
 
   /**
    * Creates an iterator of intermediate rows. The returned rows will be consumed by {@link #parseInputRows} and
-   * {@link #toMap}.
+   * {@link #toMap}. Either this or {@link #intermediateRowIteratorWithMetadata()} should be implemented
+   */
+  protected CloseableIterator<T> intermediateRowIterator() throws IOException
+  {
+    throw new UnsupportedEncodingException("intermediateRowIterator not implemented");
+  }
+
+  /**
+   * Same as {@code intermediateRowIterator}, but it also contains the metadata such as the line number to generate
+   * more informative {@link ParseException}.
+   */
+  protected CloseableIteratorWithMetadata<T> intermediateRowIteratorWithMetadata() throws IOException
+  {
+    return CloseableIteratorWithMetadata.fromCloseableIterator(intermediateRowIterator());
+  }
+
+  /**
+   * @return InputEntity which the subclass is reading from. Useful in generating informative {@link ParseException}s

Review comment:
       can you add an example here e.g. for filename can be a useful info. 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -52,28 +57,46 @@
       // good idea. Subclasses could implement read() with some duplicate codes to avoid unnecessary iteration on
       // a singleton list.
       Iterator<InputRow> rows = null;
+      long currentRecordNumber = 1;
 
       @Override
       public boolean hasNext()
       {
         if (rows == null || !rows.hasNext()) {
-          if (!intermediateRowIterator.hasNext()) {
+          if (!intermediateRowIteratorWithMetadata.hasNext()) {
             return false;
           }
-          final T row = intermediateRowIterator.next();
+          final T row = intermediateRowIteratorWithMetadata.next();
+          final Map<String, Object> metadata = intermediateRowIteratorWithMetadata.metadata();

Review comment:
       we can move it to the catch block so metadata is prepared only if needed. 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,39 +116,79 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {

Review comment:
       could we just use regular `hasNext, next()` etc and get rid of `mapWithMetadata` entirely? 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -155,6 +239,24 @@ public void close() throws IOException
    */
   protected abstract List<Map<String, Object>> toMap(T intermediateRow) throws IOException;
 
+  private String buildParseExceptionMessage(
+      String formatString,
+      @Nullable InputEntity source,
+      @Nullable Long recordNumber,
+      Map<String, Object> metadata,
+      Object... baseArgs
+  )
+  {
+    Map<String, Object> temp = Maps.newHashMap(metadata);
+    if (source != null) {
+      temp.put("source", source.getUri());
+    }
+    if (recordNumber != null) {
+      temp.put("recordNumber", recordNumber);
+    }
+    return StringUtils.nonStrictFormat(formatString, baseArgs, temp);

Review comment:
       your format string has only one '%s', right? 




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r811444450



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -155,6 +248,32 @@ public void close() throws IOException
    */
   protected abstract List<Map<String, Object>> toMap(T intermediateRow) throws IOException;
 
+  private String buildParseExceptionMessage(
+      String baseExceptionMessage,
+      @Nullable InputEntity source,
+      @Nullable Long recordNumber,
+      @Nullable Map<String, Object> metadata
+  )
+  {
+    StringBuilder sb = new StringBuilder(baseExceptionMessage);
+    List<String> temp = new ArrayList<>();
+    if (source != null && source.getUri() != null) {
+      temp.add(StringUtils.format("Source info:[%s]", source.getUri()));
+    }
+    if (recordNumber != null) {
+      temp.add(StringUtils.format("Record number:[%d]", recordNumber));
+    }
+    if (metadata != null && !metadata.isEmpty()) {
+      temp.add(StringUtils.format("Additional info:%s", metadata));

Review comment:
       Updated to seem something like
   
   ```
   org.apache.druid.java.util.common.parsers.ParseException: Timestamp[null] is unparseable! Event: {3=3, Lets do some "normal" quotes=Lets do some "normal" quotes, 2018-05-05T10:00:00Z=2018-05-05T10:... (Line: 2, Record: 1)
   ```




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on pull request #12259:
URL: https://github.com/apache/druid/pull/12259#issuecomment-1054466121


   Ignored code coverage failures since only a simple getter was missing code coverage. 


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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r806836691



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -155,6 +239,24 @@ public void close() throws IOException
    */
   protected abstract List<Map<String, Object>> toMap(T intermediateRow) throws IOException;
 
+  private String buildParseExceptionMessage(
+      String formatString,
+      @Nullable InputEntity source,
+      @Nullable Long recordNumber,
+      Map<String, Object> metadata,

Review comment:
       this can be null too I suppose. since not every implementation has metadata. 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -135,9 +198,30 @@ public void close() throws IOException
 
   /**
    * Creates an iterator of intermediate rows. The returned rows will be consumed by {@link #parseInputRows} and
-   * {@link #toMap}.
+   * {@link #toMap}. Either this or {@link #intermediateRowIteratorWithMetadata()} should be implemented
+   */
+  protected CloseableIterator<T> intermediateRowIterator() throws IOException
+  {
+    throw new UnsupportedEncodingException("intermediateRowIterator not implemented");
+  }
+
+  /**
+   * Same as {@code intermediateRowIterator}, but it also contains the metadata such as the line number to generate
+   * more informative {@link ParseException}.
+   */
+  protected CloseableIteratorWithMetadata<T> intermediateRowIteratorWithMetadata() throws IOException
+  {
+    return CloseableIteratorWithMetadata.fromCloseableIterator(intermediateRowIterator());
+  }
+
+  /**
+   * @return InputEntity which the subclass is reading from. Useful in generating informative {@link ParseException}s

Review comment:
       can you add an example here e.g. for filename can be a useful info. 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -52,28 +57,46 @@
       // good idea. Subclasses could implement read() with some duplicate codes to avoid unnecessary iteration on
       // a singleton list.
       Iterator<InputRow> rows = null;
+      long currentRecordNumber = 1;
 
       @Override
       public boolean hasNext()
       {
         if (rows == null || !rows.hasNext()) {
-          if (!intermediateRowIterator.hasNext()) {
+          if (!intermediateRowIteratorWithMetadata.hasNext()) {
             return false;
           }
-          final T row = intermediateRowIterator.next();
+          final T row = intermediateRowIteratorWithMetadata.next();
+          final Map<String, Object> metadata = intermediateRowIteratorWithMetadata.metadata();

Review comment:
       we can move it to the catch block so metadata is prepared only if needed. 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,39 +116,79 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {

Review comment:
       could we just use regular `hasNext, next()` etc and get rid of `mapWithMetadata` entirely? 

##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -155,6 +239,24 @@ public void close() throws IOException
    */
   protected abstract List<Map<String, Object>> toMap(T intermediateRow) throws IOException;
 
+  private String buildParseExceptionMessage(
+      String formatString,
+      @Nullable InputEntity source,
+      @Nullable Long recordNumber,
+      Map<String, Object> metadata,
+      Object... baseArgs
+  )
+  {
+    Map<String, Object> temp = Maps.newHashMap(metadata);
+    if (source != null) {
+      temp.put("source", source.getUri());
+    }
+    if (recordNumber != null) {
+      temp.put("recordNumber", recordNumber);
+    }
+    return StringUtils.nonStrictFormat(formatString, baseArgs, temp);

Review comment:
       your format string has only one '%s', right? 




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r806846852



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,39 +116,79 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {

Review comment:
       If we use that, then the errors generated in the sample() call won't have the additional metadata. 




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] LakshSingla commented on a change in pull request #12259: Make ParseExceptions more informative

Posted by GitBox <gi...@apache.org>.
LakshSingla commented on a change in pull request #12259:
URL: https://github.com/apache/druid/pull/12259#discussion_r806846852



##########
File path: core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java
##########
@@ -93,39 +116,79 @@ public InputRow next()
       @Override
       public void close() throws IOException
       {
-        intermediateRowIterator.close();
+        intermediateRowIteratorWithMetadata.close();
       }
     };
   }
 
   @Override
   public CloseableIterator<InputRowListPlusRawValues> sample() throws IOException
   {
-    return intermediateRowIterator().map(row -> {

Review comment:
       If we use that, then the errors generated in the sample() call won't have the additional metadata. 




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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org