You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/05/14 05:38:44 UTC

[GitHub] [arrow] n3world opened a new pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

n3world opened a new pull request #10321:
URL: https://github.com/apache/arrow/pull/10321


   For serial CSV readers track the absolute row number and report it in errors encountered during parsing or converting.
   
   I did try to get row numbers for the parallel reader but the only way I thought that could work would be to add delimiter counting to the Chunker but that seemed to add more complexity than I wanted to.


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



[GitHub] [arrow] pitrou commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r640038839



##########
File path: python/pyarrow/tests/test_csv.py
##########
@@ -52,18 +52,20 @@ def generate_col_names():
             yield first + second
 
 
-def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n'):
+def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n', write_names=True):
     arr = np.random.RandomState(42).randint(0, 1000, size=(num_cols, num_rows))
-    col_names = list(itertools.islice(generate_col_names(), num_cols))
     csv = io.StringIO()
-    csv.write(",".join(col_names))
+    if write_names:
+        col_names = list(itertools.islice(generate_col_names(), num_cols))
+        csv.write(",".join(col_names))
     csv.write(linesep)
     for row in arr.T:
         csv.write(",".join(map(str, row)))
         csv.write(linesep)
     csv = csv.getvalue().encode()
     columns = [pa.array(a, type=pa.int64()) for a in arr]
-    expected = pa.Table.from_arrays(columns, col_names)
+    expected = pa.Table.from_arrays(
+        columns, col_names) if write_names else None

Review comment:
       Well, the test should ensure that the column names are the same.




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



[GitHub] [arrow] github-actions[bot] commented on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-841020552


   https://issues.apache.org/jira/browse/ARROW-12675


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



[GitHub] [arrow] ursabot edited a comment on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-849563333


   Benchmark runs are scheduled for baseline = 861b5dad14e609f042d7f32ba3926e91e232ba92 and contender = fe786901b776e7830b71081df91e157651b8e3dc. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Scheduled] [ec2-t3-large-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/93a0fc9ee82344919718876aaa25da63...bc61a35c3cd74e59ba6889fd981f2b3b/)
   [Finished :arrow_down:75.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/3064ea0890064684b676b9ae0023d61d...03dd0355ce7449bbbaf57e62f022ad7d/)
   [Scheduled] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/808690002da441d5a964b8a3c3e95894...f178cac0047648aaa9d6ba7a4d75bd57/)
   [Scheduled] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/1f9c9be0ccb640bcae27f6d7dcfb094a...195af935006c45568bf56d92917c1038/)
   


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



[GitHub] [arrow] pitrou commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r640546470



##########
File path: cpp/src/arrow/dataset/file_csv.cc
##########
@@ -51,7 +51,7 @@ Result<std::unordered_set<std::string>> GetColumnNames(
     const csv::ParseOptions& parse_options, util::string_view first_block,
     MemoryPool* pool) {
   uint32_t parsed_size = 0;
-  csv::BlockParser parser(pool, parse_options, /*num_cols=*/-1,
+  csv::BlockParser parser(pool, parse_options, /*num_cols=*/-1, /*first_row=*/0,

Review comment:
       Should this be 1, as is done in the CSV reader?




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



[GitHub] [arrow] ursabot edited a comment on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-849563333


   Benchmark runs are scheduled for baseline = 861b5dad14e609f042d7f32ba3926e91e232ba92 and contender = fe786901b776e7830b71081df91e157651b8e3dc. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Failed :arrow_down:0.0% :arrow_up:33.33%] [ec2-t3-large-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/93a0fc9ee82344919718876aaa25da63...bc61a35c3cd74e59ba6889fd981f2b3b/)
   [Finished :arrow_down:75.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/3064ea0890064684b676b9ae0023d61d...03dd0355ce7449bbbaf57e62f022ad7d/)
   [Finished :arrow_down:2.07% :arrow_up:0.0%] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/808690002da441d5a964b8a3c3e95894...f178cac0047648aaa9d6ba7a4d75bd57/)
   [Finished :arrow_down:5.06% :arrow_up:5.61%] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/1f9c9be0ccb640bcae27f6d7dcfb094a...195af935006c45568bf56d92917c1038/)
   


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



[GitHub] [arrow] pitrou commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r639580493



##########
File path: python/pyarrow/tests/test_csv.py
##########
@@ -974,6 +976,56 @@ def read_csv(self, *args, validate_full=True, **kwargs):
         table.validate(full=validate_full)
         return table
 
+    def test_row_numbers_in_errors(self):
+        """ Row numbers are only correctly counted in serial reads """
+        csv, _ = make_random_csv(4, 100, write_names=True)
+
+        read_options = ReadOptions()
+        read_options.block_size = len(csv) / 3
+        convert_options = ConvertOptions()
+        convert_options.column_types = {"a": pa.int32(), "d": pa.int32()}
+
+        # Test without skip_rows and column names in the csv
+        csv_bad_columns = csv + b"1,2\r\n"
+        with pytest.raises(pa.ArrowInvalid,
+                           match="Row 102: Expected 4 columns, got 2"):
+            self.read_bytes(csv_bad_columns, read_options=read_options,
+                            convert_options=convert_options)
+
+        csv_bad_type = csv + b"a,b,c,d\r\n"
+        message = ("In CSV column #0: Row 102: " +

Review comment:
       This should probably be "Row #102", no?

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -319,11 +319,12 @@ class ReaderMixin {
  public:
   ReaderMixin(io::IOContext io_context, std::shared_ptr<io::InputStream> input,
               const ReadOptions& read_options, const ParseOptions& parse_options,
-              const ConvertOptions& convert_options)
+              const ConvertOptions& convert_options, bool count_rows)
       : io_context_(std::move(io_context)),
         read_options_(read_options),
         parse_options_(parse_options),
         convert_options_(convert_options),
+        num_rows_seen_(count_rows ? 1 : -1),

Review comment:
       This is weird. Why not have a separate member `bool count_rows_`?

##########
File path: python/pyarrow/tests/test_csv.py
##########
@@ -52,18 +52,20 @@ def generate_col_names():
             yield first + second
 
 
-def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n'):
+def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n', write_names=True):
     arr = np.random.RandomState(42).randint(0, 1000, size=(num_cols, num_rows))
-    col_names = list(itertools.islice(generate_col_names(), num_cols))
     csv = io.StringIO()
-    csv.write(",".join(col_names))
+    if write_names:
+        col_names = list(itertools.islice(generate_col_names(), num_cols))
+        csv.write(",".join(col_names))
     csv.write(linesep)
     for row in arr.T:
         csv.write(",".join(map(str, row)))
         csv.write(linesep)
     csv = csv.getvalue().encode()
     columns = [pa.array(a, type=pa.int64()) for a in arr]
-    expected = pa.Table.from_arrays(columns, col_names)
+    expected = pa.Table.from_arrays(
+        columns, col_names) if write_names else None

Review comment:
       I'm not sure what the condition is for here?

##########
File path: cpp/src/arrow/dataset/file_csv.cc
##########
@@ -51,7 +51,7 @@ Result<std::unordered_set<std::string>> GetColumnNames(
     const csv::ParseOptions& parse_options, util::string_view first_block,
     MemoryPool* pool) {
   uint32_t parsed_size = 0;
-  csv::BlockParser parser(pool, parse_options, /*num_cols=*/-1,
+  csv::BlockParser parser(pool, parse_options, /*num_cols=*/-1, /*num_rows=*/0,

Review comment:
       Are you sure `num_rows` is the argument name?

##########
File path: cpp/src/arrow/csv/parser.cc
##########
@@ -35,14 +35,21 @@ using detail::ParsedValueDesc;
 
 namespace {
 
-Status ParseError(const char* message) {
-  return Status::Invalid("CSV parse error: ", message);
+template <typename... Args>
+Status ParseError(Args&&... args) {
+  return Status::Invalid("CSV parse error: ", std::forward<Args>(args)...);
 }
 
-Status MismatchingColumns(int32_t expected, int32_t actual) {
-  char s[50];
-  snprintf(s, sizeof(s), "Expected %d columns, got %d", expected, actual);
-  return ParseError(s);
+Status MismatchingColumns(int32_t expected, int32_t actual, int64_t row_num,
+                          util::string_view row) {
+  if (row.length() > 100) {
+    row = row.substr(0, 96);

Review comment:
       It probably isn't great to show a truncated row without mentioning that it is truncated.

##########
File path: cpp/src/arrow/csv/parser.h
##########
@@ -63,19 +63,24 @@ class ARROW_EXPORT DataBatch {
   uint32_t num_bytes() const { return parsed_size_; }
 
   template <typename Visitor>
-  Status VisitColumn(int32_t col_index, Visitor&& visit) const {
+  Status VisitColumn(int32_t col_index, int64_t first_row, Visitor&& visit) const {
     using detail::ParsedValueDesc;
 
+    int64_t row = first_row;
     for (size_t buf_index = 0; buf_index < values_buffers_.size(); ++buf_index) {
       const auto& values_buffer = values_buffers_[buf_index];
       const auto values = reinterpret_cast<const ParsedValueDesc*>(values_buffer->data());
       const auto max_pos =
           static_cast<int32_t>(values_buffer->size() / sizeof(ParsedValueDesc)) - 1;
-      for (int32_t pos = col_index; pos < max_pos; pos += num_cols_) {
+      for (int32_t pos = col_index; pos < max_pos; pos += num_cols_, ++row) {
         auto start = values[pos].offset;
         auto stop = values[pos + 1].offset;
         auto quoted = values[pos + 1].quoted;
-        ARROW_RETURN_NOT_OK(visit(parsed_ + start, stop - start, quoted));
+        Status status = visit(parsed_ + start, stop - start, quoted);
+        if (ARROW_PREDICT_FALSE(first_row >= 0 && !status.ok())) {

Review comment:
       Suggestion:
   ```c++
   if (ARROW_PREDICT_FALSE(!status.ok())) {
     if (first_row >= 0) {
       status = ...
     }
     return status;
   }
   ```




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



[GitHub] [arrow] lidavidm commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r635534220



##########
File path: cpp/src/arrow/csv/parser_test.cc
##########
@@ -536,22 +537,42 @@ TEST(BlockParser, QuotesSpecial) {
 TEST(BlockParser, MismatchingNumColumns) {
   uint32_t out_size;
   {
-    BlockParser parser(ParseOptions::Defaults());
+    BlockParser parser(ParseOptions::Defaults(), -1, 0 /* first_row */);
     auto csv = MakeCSVData({"a,b\nc\n"});
     Status st = Parse(parser, csv, &out_size);
     ASSERT_RAISES(Invalid, st);
+    ASSERT_NE(std::string::npos,
+              st.message().find("CSV parse error: Row 1: Expected 2 columns, got 1: c"))
+        << st.message();
   }
   {
-    BlockParser parser(ParseOptions::Defaults(), 2 /* num_cols */);
+    BlockParser parser(ParseOptions::Defaults(), 2 /* num_cols */, 0 /* first_row */);
     auto csv = MakeCSVData({"a\n"});
     Status st = Parse(parser, csv, &out_size);
     ASSERT_RAISES(Invalid, st);
+    ASSERT_NE(std::string::npos,
+              st.message().find("CSV parse error: Row 0: Expected 2 columns, got 1: a"))
+        << st.message();
   }
   {
-    BlockParser parser(ParseOptions::Defaults(), 2 /* num_cols */);
+    BlockParser parser(ParseOptions::Defaults(), 2 /* num_cols */, 50 /* first_row */);
     auto csv = MakeCSVData({"a,b,c\n"});
     Status st = Parse(parser, csv, &out_size);
     ASSERT_RAISES(Invalid, st);
+    ASSERT_NE(
+        std::string::npos,
+        st.message().find("CSV parse error: Row 50: Expected 2 columns, got 3: a,b,c"))
+        << st.message();
+  }
+  // No row number
+  {
+    BlockParser parser(ParseOptions::Defaults(), 2 /* num_cols */, -1);
+    auto csv = MakeCSVData({"a\n"});
+    Status st = Parse(parser, csv, &out_size);
+    ASSERT_RAISES(Invalid, st);
+    ASSERT_NE(std::string::npos,
+              st.message().find("CSV parse error: Expected 2 columns, got 1: a"))

Review comment:
       Sorry, just a nit, you can use gmock in this block of tests too.




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



[GitHub] [arrow] pitrou commented on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-849653226


   I've rebased on master to try and fix the AppVeyor CI issue.


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



[GitHub] [arrow] pitrou commented on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-849563094


   @ursabot please benchmark


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



[GitHub] [arrow] pitrou commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r639829796



##########
File path: python/pyarrow/tests/test_csv.py
##########
@@ -52,18 +52,20 @@ def generate_col_names():
             yield first + second
 
 
-def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n'):
+def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n', write_names=True):
     arr = np.random.RandomState(42).randint(0, 1000, size=(num_cols, num_rows))
-    col_names = list(itertools.islice(generate_col_names(), num_cols))
     csv = io.StringIO()
-    csv.write(",".join(col_names))
+    if write_names:
+        col_names = list(itertools.islice(generate_col_names(), num_cols))
+        csv.write(",".join(col_names))
     csv.write(linesep)
     for row in arr.T:
         csv.write(",".join(map(str, row)))
         csv.write(linesep)
     csv = csv.getvalue().encode()
     columns = [pa.array(a, type=pa.int64()) for a in arr]
-    expected = pa.Table.from_arrays(columns, col_names)
+    expected = pa.Table.from_arrays(
+        columns, col_names) if write_names else None

Review comment:
       Or you can set `col_names` unconditionally, which will be less fragile IMO.




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



[GitHub] [arrow] n3world commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
n3world commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r639783428



##########
File path: python/pyarrow/tests/test_csv.py
##########
@@ -52,18 +52,20 @@ def generate_col_names():
             yield first + second
 
 
-def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n'):
+def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n', write_names=True):
     arr = np.random.RandomState(42).randint(0, 1000, size=(num_cols, num_rows))
-    col_names = list(itertools.islice(generate_col_names(), num_cols))
     csv = io.StringIO()
-    csv.write(",".join(col_names))
+    if write_names:
+        col_names = list(itertools.islice(generate_col_names(), num_cols))
+        csv.write(",".join(col_names))
     csv.write(linesep)
     for row in arr.T:
         csv.write(",".join(map(str, row)))
         csv.write(linesep)
     csv = csv.getvalue().encode()
     columns = [pa.array(a, type=pa.int64()) for a in arr]
-    expected = pa.Table.from_arrays(columns, col_names)
+    expected = pa.Table.from_arrays(
+        columns, col_names) if write_names else None

Review comment:
       If write_names is false then col_names is not set so the Table cannot be created. I'll change this to be a condition on col_names because that is a bit more obvious

##########
File path: cpp/src/arrow/csv/parser.cc
##########
@@ -35,14 +35,21 @@ using detail::ParsedValueDesc;
 
 namespace {
 
-Status ParseError(const char* message) {
-  return Status::Invalid("CSV parse error: ", message);
+template <typename... Args>
+Status ParseError(Args&&... args) {
+  return Status::Invalid("CSV parse error: ", std::forward<Args>(args)...);
 }
 
-Status MismatchingColumns(int32_t expected, int32_t actual) {
-  char s[50];
-  snprintf(s, sizeof(s), "Expected %d columns, got %d", expected, actual);
-  return ParseError(s);
+Status MismatchingColumns(int32_t expected, int32_t actual, int64_t row_num,
+                          util::string_view row) {
+  if (row.length() > 100) {
+    row = row.substr(0, 96);

Review comment:
       Sorry about that the ellipse was accidentally removed when I added the row_num to the output

##########
File path: cpp/src/arrow/csv/parser.h
##########
@@ -63,19 +63,24 @@ class ARROW_EXPORT DataBatch {
   uint32_t num_bytes() const { return parsed_size_; }
 
   template <typename Visitor>
-  Status VisitColumn(int32_t col_index, Visitor&& visit) const {
+  Status VisitColumn(int32_t col_index, int64_t first_row, Visitor&& visit) const {
     using detail::ParsedValueDesc;
 
+    int64_t row = first_row;
     for (size_t buf_index = 0; buf_index < values_buffers_.size(); ++buf_index) {
       const auto& values_buffer = values_buffers_[buf_index];
       const auto values = reinterpret_cast<const ParsedValueDesc*>(values_buffer->data());
       const auto max_pos =
           static_cast<int32_t>(values_buffer->size() / sizeof(ParsedValueDesc)) - 1;
-      for (int32_t pos = col_index; pos < max_pos; pos += num_cols_) {
+      for (int32_t pos = col_index; pos < max_pos; pos += num_cols_, ++row) {
         auto start = values[pos].offset;
         auto stop = values[pos + 1].offset;
         auto quoted = values[pos + 1].quoted;
-        ARROW_RETURN_NOT_OK(visit(parsed_ + start, stop - start, quoted));
+        Status status = visit(parsed_ + start, stop - start, quoted);
+        if (ARROW_PREDICT_FALSE(first_row >= 0 && !status.ok())) {

Review comment:
       ARROW_RETURN_NOT_OK adds the extra context when that is enabled so I think it would be better to keep that around or add a new macro which doesn't check status just adds the context and returns when it is enabled

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -319,11 +319,12 @@ class ReaderMixin {
  public:
   ReaderMixin(io::IOContext io_context, std::shared_ptr<io::InputStream> input,
               const ReadOptions& read_options, const ParseOptions& parse_options,
-              const ConvertOptions& convert_options)
+              const ConvertOptions& convert_options, bool count_rows)
       : io_context_(std::move(io_context)),
         read_options_(read_options),
         parse_options_(parse_options),
         convert_options_(convert_options),
+        num_rows_seen_(count_rows ? 1 : -1),

Review comment:
       Just to reduce the number of member variables. Using a separate bool might be a bit clearer but it doesn't simplify the code any and you end up having one variable indicating if another variable is being used and using -1/<0 to indicate disabled is common enough I didn't think it obfuscated the intent too much.
   
   If you feel strongly there should be two member variables to track row count I can make that 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:
users@infra.apache.org



[GitHub] [arrow] lidavidm commented on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-845117528


   Antoine, could you take a quick look here as you're more familiar with the CSV parser? The changes here look minimally invasive + having the extra error context would be nice.


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



[GitHub] [arrow] n3world commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
n3world commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r640037240



##########
File path: python/pyarrow/tests/test_csv.py
##########
@@ -52,18 +52,20 @@ def generate_col_names():
             yield first + second
 
 
-def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n'):
+def make_random_csv(num_cols=2, num_rows=10, linesep='\r\n', write_names=True):
     arr = np.random.RandomState(42).randint(0, 1000, size=(num_cols, num_rows))
-    col_names = list(itertools.islice(generate_col_names(), num_cols))
     csv = io.StringIO()
-    csv.write(",".join(col_names))
+    if write_names:
+        col_names = list(itertools.islice(generate_col_names(), num_cols))
+        csv.write(",".join(col_names))
     csv.write(linesep)
     for row in arr.T:
         csv.write(",".join(map(str, row)))
         csv.write(linesep)
     csv = csv.getvalue().encode()
     columns = [pa.array(a, type=pa.int64()) for a in arr]
-    expected = pa.Table.from_arrays(columns, col_names)
+    expected = pa.Table.from_arrays(
+        columns, col_names) if write_names else None

Review comment:
       But then the column names returned by the csv parser may be different than the table or is that not an issue?




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



[GitHub] [arrow] pitrou closed pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou closed pull request #10321:
URL: https://github.com/apache/arrow/pull/10321


   


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



[GitHub] [arrow] ursabot edited a comment on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-849563333


   Benchmark runs are scheduled for baseline = 861b5dad14e609f042d7f32ba3926e91e232ba92 and contender = fe786901b776e7830b71081df91e157651b8e3dc. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Failed :arrow_down:0.0% :arrow_up:33.33%] [ec2-t3-large-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/93a0fc9ee82344919718876aaa25da63...bc61a35c3cd74e59ba6889fd981f2b3b/)
   [Finished :arrow_down:75.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/3064ea0890064684b676b9ae0023d61d...03dd0355ce7449bbbaf57e62f022ad7d/)
   [Scheduled] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/808690002da441d5a964b8a3c3e95894...f178cac0047648aaa9d6ba7a4d75bd57/)
   [Finished :arrow_down:5.06% :arrow_up:5.61%] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/1f9c9be0ccb640bcae27f6d7dcfb094a...195af935006c45568bf56d92917c1038/)
   


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



[GitHub] [arrow] pitrou commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r639830892



##########
File path: cpp/src/arrow/csv/parser.h
##########
@@ -63,19 +63,24 @@ class ARROW_EXPORT DataBatch {
   uint32_t num_bytes() const { return parsed_size_; }
 
   template <typename Visitor>
-  Status VisitColumn(int32_t col_index, Visitor&& visit) const {
+  Status VisitColumn(int32_t col_index, int64_t first_row, Visitor&& visit) const {
     using detail::ParsedValueDesc;
 
+    int64_t row = first_row;
     for (size_t buf_index = 0; buf_index < values_buffers_.size(); ++buf_index) {
       const auto& values_buffer = values_buffers_[buf_index];
       const auto values = reinterpret_cast<const ParsedValueDesc*>(values_buffer->data());
       const auto max_pos =
           static_cast<int32_t>(values_buffer->size() / sizeof(ParsedValueDesc)) - 1;
-      for (int32_t pos = col_index; pos < max_pos; pos += num_cols_) {
+      for (int32_t pos = col_index; pos < max_pos; pos += num_cols_, ++row) {
         auto start = values[pos].offset;
         auto stop = values[pos + 1].offset;
         auto quoted = values[pos + 1].quoted;
-        ARROW_RETURN_NOT_OK(visit(parsed_ + start, stop - start, quoted));
+        Status status = visit(parsed_ + start, stop - start, quoted);
+        if (ARROW_PREDICT_FALSE(first_row >= 0 && !status.ok())) {

Review comment:
       Ah, you're right. But you should still ensure that `!status.ok()` is the first condition inside `ARROW_PREDICT_FALSE`.




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



[GitHub] [arrow] pitrou commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r639830246



##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -319,11 +319,12 @@ class ReaderMixin {
  public:
   ReaderMixin(io::IOContext io_context, std::shared_ptr<io::InputStream> input,
               const ReadOptions& read_options, const ParseOptions& parse_options,
-              const ConvertOptions& convert_options)
+              const ConvertOptions& convert_options, bool count_rows)
       : io_context_(std::move(io_context)),
         read_options_(read_options),
         parse_options_(parse_options),
         convert_options_(convert_options),
+        num_rows_seen_(count_rows ? 1 : -1),

Review comment:
       Having separate member variables would be clearer IMO.




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



[GitHub] [arrow] lidavidm commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r635318248



##########
File path: python/pyarrow/tests/test_csv.py
##########
@@ -974,6 +976,36 @@ def read_csv(self, *args, validate_full=True, **kwargs):
         table.validate(full=validate_full)
         return table
 
+    def test_row_numbers_in_errors(self):
+        """ Row numbers are only correctly counted in serial reads """
+        csv, _ = make_random_csv(4, 100, write_names=False)

Review comment:
       Is there a particular reason to only test without the header row? It'd be good to check that that gets handled properly too.

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -1004,7 +1016,7 @@ Future<std::shared_ptr<StreamingReader>> MakeStreamingReader(
     const ParseOptions& parse_options, const ConvertOptions& convert_options) {
   std::shared_ptr<BaseStreamingReader> reader;
   reader = std::make_shared<SerialStreamingReader>(
-      io_context, cpu_executor, input, read_options, parse_options, convert_options);
+      io_context, cpu_executor, input, read_options, parse_options, convert_options, true);

Review comment:
       ```suggestion
         io_context, cpu_executor, input, read_options, parse_options, convert_options, /*count_rows=*/true);
   ```

##########
File path: cpp/src/arrow/csv/parser_test.cc
##########
@@ -623,5 +643,48 @@ TEST(BlockParser, QuotedEscape) {
   }
 }
 
+TEST(BlockParser, RowNumberAppendedToError) {
+  auto options = ParseOptions::Defaults();
+  auto csv = "a,b,c\nd,e,f\ng,h,i\n";
+  {
+    BlockParser parser(options, -1, 0);
+    ASSERT_NO_FATAL_FAILURE(AssertParseOk(parser, csv));
+    int row = 0;
+    auto status = parser.VisitColumn(
+        0, [row](const uint8_t* data, uint32_t size, bool quoted) mutable -> Status {
+          return ++row == 2 ? Status::Invalid("Bad value") : Status::OK();
+        });
+    ASSERT_RAISES(Invalid, status);
+    ASSERT_NE(std::string::npos, status.message().find("Row 1: Bad value"))
+        << status.message();

Review comment:
       You can do something like `EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, ::testing::HasSubstr("Row 1: Bad value", status)`

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -888,9 +899,10 @@ class AsyncThreadedTableReader
                            std::shared_ptr<io::InputStream> input,
                            const ReadOptions& read_options,
                            const ParseOptions& parse_options,
-                           const ConvertOptions& convert_options, Executor* cpu_executor)
+                           const ConvertOptions& convert_options,
+                           Executor* cpu_executor)
       : BaseTableReader(std::move(io_context), input, read_options, parse_options,
-                        convert_options),
+                        convert_options, false),

Review comment:
       ```suggestion
                           convert_options, /*count_rows=*/false),
   ```

##########
File path: cpp/src/arrow/csv/parser_test.cc
##########
@@ -623,5 +643,48 @@ TEST(BlockParser, QuotedEscape) {
   }
 }
 
+TEST(BlockParser, RowNumberAppendedToError) {
+  auto options = ParseOptions::Defaults();
+  auto csv = "a,b,c\nd,e,f\ng,h,i\n";
+  {
+    BlockParser parser(options, -1, 0);
+    ASSERT_NO_FATAL_FAILURE(AssertParseOk(parser, csv));
+    int row = 0;
+    auto status = parser.VisitColumn(
+        0, [row](const uint8_t* data, uint32_t size, bool quoted) mutable -> Status {
+          return ++row == 2 ? Status::Invalid("Bad value") : Status::OK();
+        });
+    ASSERT_RAISES(Invalid, status);
+    ASSERT_NE(std::string::npos, status.message().find("Row 1: Bad value"))
+        << status.message();
+  }
+
+  {
+    BlockParser parser(options, -1, 100);
+    ASSERT_NO_FATAL_FAILURE(AssertParseOk(parser, csv));
+    int row = 0;
+    auto status = parser.VisitColumn(
+        0, [row](const uint8_t* data, uint32_t size, bool quoted) mutable -> Status {
+          return ++row == 3 ? Status::Invalid("Bad value") : Status::OK();
+        });
+    ASSERT_RAISES(Invalid, status);
+    ASSERT_NE(std::string::npos, status.message().find("Row 102: Bad value"))
+        << status.message();
+  }
+
+  // No first row specified should not append row information
+  {
+    BlockParser parser(options, -1, -1);
+    ASSERT_NO_FATAL_FAILURE(AssertParseOk(parser, csv));
+    int row = 0;
+    auto status = parser.VisitColumn(
+        0, [row](const uint8_t* data, uint32_t size, bool quoted) mutable -> Status {
+          return ++row == 3 ? Status::Invalid("Bad value") : Status::OK();
+        });
+    ASSERT_RAISES(Invalid, status);
+    ASSERT_EQ(std::string::npos, status.message().find("Row")) << status.message();

Review comment:
       And here you can use `::testing::Not(::testing::HasSubstr())`

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -888,9 +899,10 @@ class AsyncThreadedTableReader
                            std::shared_ptr<io::InputStream> input,
                            const ReadOptions& read_options,
                            const ParseOptions& parse_options,
-                           const ConvertOptions& convert_options, Executor* cpu_executor)
+                           const ConvertOptions& convert_options,
+                           Executor* cpu_executor)
       : BaseTableReader(std::move(io_context), input, read_options, parse_options,
-                        convert_options),
+                        convert_options, false),

Review comment:
       (and maybe note that the parallel reader cannot count rows yet)

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -500,6 +510,7 @@ class ReaderMixin {
 
   // Number of columns in the CSV file
   int32_t num_csv_cols_ = -1;
+  int64_t num_rows_;

Review comment:
       nit: maybe something more like `last_row_number_` or `num_seen_rows_` and a comment noting `-1` means we aren't tracking the row count?

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -992,7 +1004,7 @@ Result<std::shared_ptr<TableReader>> MakeTableReader(
         io_context, input, read_options, parse_options, convert_options, cpu_executor);
   } else {
     reader = std::make_shared<SerialTableReader>(io_context, input, read_options,
-                                                 parse_options, convert_options);
+                                                 parse_options, convert_options, true);

Review comment:
       ```suggestion
                                                    parse_options, convert_options, /*count_rows=*/true);
   ```

##########
File path: cpp/src/arrow/csv/reader.cc
##########
@@ -1004,7 +1016,7 @@ Future<std::shared_ptr<StreamingReader>> MakeStreamingReader(
     const ParseOptions& parse_options, const ConvertOptions& convert_options) {
   std::shared_ptr<BaseStreamingReader> reader;
   reader = std::make_shared<SerialStreamingReader>(
-      io_context, cpu_executor, input, read_options, parse_options, convert_options);
+      io_context, cpu_executor, input, read_options, parse_options, convert_options, true);

Review comment:
       (will probably need to be reformatted 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.

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



[GitHub] [arrow] n3world commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
n3world commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r640039477



##########
File path: cpp/src/arrow/csv/parser.h
##########
@@ -63,19 +63,24 @@ class ARROW_EXPORT DataBatch {
   uint32_t num_bytes() const { return parsed_size_; }
 
   template <typename Visitor>
-  Status VisitColumn(int32_t col_index, Visitor&& visit) const {
+  Status VisitColumn(int32_t col_index, int64_t first_row, Visitor&& visit) const {
     using detail::ParsedValueDesc;
 
+    int64_t row = first_row;
     for (size_t buf_index = 0; buf_index < values_buffers_.size(); ++buf_index) {
       const auto& values_buffer = values_buffers_[buf_index];
       const auto values = reinterpret_cast<const ParsedValueDesc*>(values_buffer->data());
       const auto max_pos =
           static_cast<int32_t>(values_buffer->size() / sizeof(ParsedValueDesc)) - 1;
-      for (int32_t pos = col_index; pos < max_pos; pos += num_cols_) {
+      for (int32_t pos = col_index; pos < max_pos; pos += num_cols_, ++row) {
         auto start = values[pos].offset;
         auto stop = values[pos + 1].offset;
         auto quoted = values[pos + 1].quoted;
-        ARROW_RETURN_NOT_OK(visit(parsed_ + start, stop - start, quoted));
+        Status status = visit(parsed_ + start, stop - start, quoted);
+        if (ARROW_PREDICT_FALSE(first_row >= 0 && !status.ok())) {

Review comment:
       I updated it to be almost what you suggested but I kept the ARROW_RETURN_NOT_OK around status. If there is a desire to add an ARROW_RETURN_WITh_CONTEXT macro I am happy to add that and modify the other macros to use it. It only gets rid of a duplicate status.ok() check which isn't much overhead so probably not really worth 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:
users@infra.apache.org



[GitHub] [arrow] ursabot commented on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
ursabot commented on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-849563333


   Benchmark runs are scheduled for baseline = 861b5dad14e609f042d7f32ba3926e91e232ba92 and contender = fe786901b776e7830b71081df91e157651b8e3dc. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Scheduled] [ec2-t3-large-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/93a0fc9ee82344919718876aaa25da63...bc61a35c3cd74e59ba6889fd981f2b3b/)
   [Scheduled] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/3064ea0890064684b676b9ae0023d61d...03dd0355ce7449bbbaf57e62f022ad7d/)
   [Scheduled] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/808690002da441d5a964b8a3c3e95894...f178cac0047648aaa9d6ba7a4d75bd57/)
   [Scheduled] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/1f9c9be0ccb640bcae27f6d7dcfb094a...195af935006c45568bf56d92917c1038/)
   


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



[GitHub] [arrow] n3world commented on a change in pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
n3world commented on a change in pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#discussion_r635433863



##########
File path: python/pyarrow/tests/test_csv.py
##########
@@ -974,6 +976,36 @@ def read_csv(self, *args, validate_full=True, **kwargs):
         table.validate(full=validate_full)
         return table
 
+    def test_row_numbers_in_errors(self):
+        """ Row numbers are only correctly counted in serial reads """
+        csv, _ = make_random_csv(4, 100, write_names=False)

Review comment:
       I did that to work with the skip rows tests so that I knew what the row names were otherwise the row names are random values. I'll add some which do not skip rows and uses the headers from the csv.

##########
File path: cpp/src/arrow/csv/parser_test.cc
##########
@@ -623,5 +643,48 @@ TEST(BlockParser, QuotedEscape) {
   }
 }
 
+TEST(BlockParser, RowNumberAppendedToError) {
+  auto options = ParseOptions::Defaults();
+  auto csv = "a,b,c\nd,e,f\ng,h,i\n";
+  {
+    BlockParser parser(options, -1, 0);
+    ASSERT_NO_FATAL_FAILURE(AssertParseOk(parser, csv));
+    int row = 0;
+    auto status = parser.VisitColumn(
+        0, [row](const uint8_t* data, uint32_t size, bool quoted) mutable -> Status {
+          return ++row == 2 ? Status::Invalid("Bad value") : Status::OK();
+        });
+    ASSERT_RAISES(Invalid, status);
+    ASSERT_NE(std::string::npos, status.message().find("Row 1: Bad value"))
+        << status.message();
+  }
+
+  {
+    BlockParser parser(options, -1, 100);
+    ASSERT_NO_FATAL_FAILURE(AssertParseOk(parser, csv));
+    int row = 0;
+    auto status = parser.VisitColumn(
+        0, [row](const uint8_t* data, uint32_t size, bool quoted) mutable -> Status {
+          return ++row == 3 ? Status::Invalid("Bad value") : Status::OK();
+        });
+    ASSERT_RAISES(Invalid, status);
+    ASSERT_NE(std::string::npos, status.message().find("Row 102: Bad value"))
+        << status.message();
+  }
+
+  // No first row specified should not append row information
+  {
+    BlockParser parser(options, -1, -1);
+    ASSERT_NO_FATAL_FAILURE(AssertParseOk(parser, csv));
+    int row = 0;
+    auto status = parser.VisitColumn(
+        0, [row](const uint8_t* data, uint32_t size, bool quoted) mutable -> Status {
+          return ++row == 3 ? Status::Invalid("Bad value") : Status::OK();
+        });
+    ASSERT_RAISES(Invalid, status);
+    ASSERT_EQ(std::string::npos, status.message().find("Row")) << status.message();

Review comment:
       Sorry I didn't know gmock was used by arrow so I only used gtest features




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



[GitHub] [arrow] ursabot edited a comment on pull request #10321: ARROW-12675: [C++] CSV parsing report row on which error occurred

Posted by GitBox <gi...@apache.org>.
ursabot edited a comment on pull request #10321:
URL: https://github.com/apache/arrow/pull/10321#issuecomment-849563333


   Benchmark runs are scheduled for baseline = 861b5dad14e609f042d7f32ba3926e91e232ba92 and contender = fe786901b776e7830b71081df91e157651b8e3dc. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Failed :arrow_down:0.0% :arrow_up:33.33%] [ec2-t3-large-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/93a0fc9ee82344919718876aaa25da63...bc61a35c3cd74e59ba6889fd981f2b3b/)
   [Finished :arrow_down:75.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/3064ea0890064684b676b9ae0023d61d...03dd0355ce7449bbbaf57e62f022ad7d/)
   [Scheduled] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/808690002da441d5a964b8a3c3e95894...f178cac0047648aaa9d6ba7a4d75bd57/)
   [Scheduled] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/1f9c9be0ccb640bcae27f6d7dcfb094a...195af935006c45568bf56d92917c1038/)
   


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