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/26 10:10:16 UTC

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

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