You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2018/11/09 23:06:21 UTC

[arrow] branch master updated: ARROW-3700: [C++] Ignore empty lines in CSV files

This is an automated email from the ASF dual-hosted git repository.

wesm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 3cf8f35  ARROW-3700: [C++] Ignore empty lines in CSV files
3cf8f35 is described below

commit 3cf8f355e1268dd8761b99719ab09cc20d372185
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Fri Nov 9 18:06:12 2018 -0500

    ARROW-3700: [C++] Ignore empty lines in CSV files
    
    The option is configurable, as it may be legitimate to parse empty lines as null values in a single-column CSV file (though that's probably uncommon).
    
    Author: Antoine Pitrou <an...@python.org>
    
    Closes #2907 from pitrou/ARROW-3700-csv-empty-lines and squashes the following commits:
    
    3626fdbc4 <Antoine Pitrou> ARROW-3700:  Ignore empty lines in CSV files
---
 cpp/src/arrow/csv/csv-parser-test.cc | 51 +++++++++++++++++++++++++++++++-----
 cpp/src/arrow/csv/options.h          |  5 +++-
 cpp/src/arrow/csv/parser.cc          | 49 +++++++++++++++++++++++++++-------
 cpp/src/arrow/csv/parser.h           |  2 ++
 cpp/src/arrow/csv/test-common.h      | 14 +++++++---
 python/pyarrow/_csv.pyx              | 22 +++++++++++++++-
 python/pyarrow/includes/libarrow.pxd |  1 +
 python/pyarrow/tests/test_csv.py     | 18 ++++++++-----
 8 files changed, 135 insertions(+), 27 deletions(-)

diff --git a/cpp/src/arrow/csv/csv-parser-test.cc b/cpp/src/arrow/csv/csv-parser-test.cc
index 55c69fb..d64e0af 100644
--- a/cpp/src/arrow/csv/csv-parser-test.cc
+++ b/cpp/src/arrow/csv/csv-parser-test.cc
@@ -131,19 +131,22 @@ TEST(BlockParser, Basics) {
   AssertColumnsEq(parser, {{"ab", "ef", ""}, {"cd", "", "ij"}, {"", "gh", "kl"}});
 }
 
-TEST(BlockParser, Empty) {
+TEST(BlockParser, EmptyHeader) {
+  // Cannot infer number of columns
+  uint32_t out_size;
   {
-    auto csv = MakeCSVData({"\n"});
+    auto csv = MakeCSVData({""});
     BlockParser parser(ParseOptions::Defaults());
-    AssertParseOk(parser, csv);
-    AssertColumnsEq(parser, {{""}});
+    ASSERT_RAISES(Invalid, ParseFinal(parser, csv, &out_size));
   }
   {
-    auto csv = MakeCSVData({"\n\n"});
+    auto csv = MakeCSVData({"\n"});
     BlockParser parser(ParseOptions::Defaults());
-    AssertParseOk(parser, csv);
-    AssertColumnsEq(parser, {{"", ""}});
+    ASSERT_RAISES(Invalid, ParseFinal(parser, csv, &out_size));
   }
+}
+
+TEST(BlockParser, Empty) {
   {
     auto csv = MakeCSVData({",\n"});
     BlockParser parser(ParseOptions::Defaults());
@@ -188,6 +191,40 @@ TEST(BlockParser, MaxNumRows) {
   AssertColumnsEq(parser, {{}});
 }
 
+TEST(BlockParser, EmptyLinesWithOneColumn) {
+  auto csv = MakeCSVData({"a\n", "\n", "b\r", "\r", "c\r\n", "\r\n", "d\n"});
+  {
+    BlockParser parser(ParseOptions::Defaults());
+    AssertParseOk(parser, csv);
+    AssertColumnsEq(parser, {{"a", "b", "c", "d"}});
+  }
+  {
+    auto options = ParseOptions::Defaults();
+    options.ignore_empty_lines = false;
+    BlockParser parser(options);
+    AssertParseOk(parser, csv);
+    AssertColumnsEq(parser, {{"a", "", "b", "", "c", "", "d"}});
+  }
+}
+
+TEST(BlockParser, EmptyLinesWithSeveralColumns) {
+  uint32_t out_size;
+  auto csv = MakeCSVData({"a,b\n", "\n", "c,d\r", "\r", "e,f\r\n", "\r\n", "g,h\n"});
+  {
+    BlockParser parser(ParseOptions::Defaults());
+    AssertParseOk(parser, csv);
+    AssertColumnsEq(parser, {{"a", "c", "e", "g"}, {"b", "d", "f", "h"}});
+  }
+  {
+    // A non-ignored empty line is a single value, but two columns are expected
+    auto options = ParseOptions::Defaults();
+    options.ignore_empty_lines = false;
+    BlockParser parser(options);
+    Status st = Parse(parser, csv, &out_size);
+    ASSERT_RAISES(Invalid, st);
+  }
+}
+
 TEST(BlockParser, TruncatedData) {
   BlockParser parser(ParseOptions::Defaults());
   auto csv = MakeCSVData({"a,b\n", "c,d\n"});
diff --git a/cpp/src/arrow/csv/options.h b/cpp/src/arrow/csv/options.h
index 0e42341..0fdde72 100644
--- a/cpp/src/arrow/csv/options.h
+++ b/cpp/src/arrow/csv/options.h
@@ -42,9 +42,12 @@ struct ARROW_EXPORT ParseOptions {
   char escape_char = '\\';
   // Whether values are allowed to contain CR (0x0d) and LF (0x0a) characters
   bool newlines_in_values = false;
+  // Whether empty lines are ignored.  If false, an empty line represents
+  // a single empty value (assuming a one-column CSV file).
+  bool ignore_empty_lines = true;
 
   // XXX Should this be in ReadOptions?
-  // Number of header rows to skip
+  // Number of header rows to skip (including the first row containing column names)
   int32_t header_rows = 1;
 
   static ParseOptions Defaults();
diff --git a/cpp/src/arrow/csv/parser.cc b/cpp/src/arrow/csv/parser.cc
index ea602df..fe7f841 100644
--- a/cpp/src/arrow/csv/parser.cc
+++ b/cpp/src/arrow/csv/parser.cc
@@ -41,6 +41,8 @@ static Status MismatchingColumns(int32_t expected, int32_t actual) {
   return ParseError(s);
 }
 
+static inline bool IsControlChar(uint8_t c) { return c < ' '; }
+
 template <bool Quoting, bool Escaping>
 class SpecializedOptions {
  public:
@@ -75,6 +77,7 @@ class BlockParser::PresizedParsedWriter {
 
   int64_t size() { return parsed_size_; }
 
+ protected:
   std::shared_ptr<ResizableBuffer> parsed_buffer_;
   uint8_t* parsed_;
   int64_t parsed_size_;
@@ -193,15 +196,31 @@ Status BlockParser::ParseLine(ValuesWriter* values_writer, ParsedWriter* parsed_
   int32_t num_cols = 0;
   char c;
 
-  values_writer->BeginLine();
-  parsed_writer->BeginLine();
+  DCHECK_GT(data_end, data);
 
   auto FinishField = [&]() { values_writer->FinishField(parsed_writer); };
 
-  DCHECK_GT(data_end, data);
+  values_writer->BeginLine();
+  parsed_writer->BeginLine();
 
   // The parsing state machine
 
+  // Special case empty lines: do we start with a newline separator?
+  c = *data;
+  if (ARROW_PREDICT_FALSE(IsControlChar(c)) && options_.ignore_empty_lines) {
+    if (c == '\r') {
+      data++;
+      if (data < data_end && *data == '\n') {
+        data++;
+      }
+      goto EmptyLine;
+    }
+    if (c == '\n') {
+      data++;
+      goto EmptyLine;
+    }
+  }
+
 FieldStart:
   // At the start of a field
   // Quoting is only recognized at start of field
@@ -231,7 +250,7 @@ InField:
   if (ARROW_PREDICT_FALSE(c == options_.delimiter)) {
     goto FieldEnd;
   }
-  if (ARROW_PREDICT_FALSE(c < ' ')) {
+  if (ARROW_PREDICT_FALSE(IsControlChar(c))) {
     if (c == '\r') {
       // In the middle of a newline separator?
       if (ARROW_PREDICT_TRUE(data < data_end) && *data == '\n') {
@@ -286,11 +305,14 @@ LineEnd:
   // At the end of line
   FinishField();
   ++num_cols;
-  if (ARROW_PREDICT_FALSE(num_cols_ == -1)) {
-    num_cols_ = num_cols;
-  } else if (ARROW_PREDICT_FALSE(num_cols != num_cols_)) {
-    return MismatchingColumns(num_cols_, num_cols);
+  if (ARROW_PREDICT_FALSE(num_cols != num_cols_)) {
+    if (num_cols_ == -1) {
+      num_cols_ = num_cols;
+    } else {
+      return MismatchingColumns(num_cols_, num_cols);
+    }
   }
+  ++num_rows_;
   *out_data = data;
   return Status::OK();
 
@@ -304,6 +326,7 @@ AbortLine:
     } else if (num_cols != num_cols_) {
       return MismatchingColumns(num_cols_, num_cols);
     }
+    ++num_rows_;
     *out_data = data;
     return Status::OK();
   }
@@ -311,6 +334,10 @@ AbortLine:
   values_writer->RollbackLine();
   parsed_writer->RollbackLine();
   return Status::OK();
+
+EmptyLine:
+  *out_data = data;
+  return Status::OK();
 }
 
 template <typename SpecializedOptions, typename ValuesWriter, typename ParsedWriter>
@@ -328,7 +355,8 @@ Status BlockParser::ParseChunk(ValuesWriter* values_writer, ParsedWriter* parsed
       break;
     }
     data = line_end;
-    ++num_rows_;
+    // This will pessimize chunk size a bit if there are empty lines,
+    // but that shouldn't be important
     --rows_in_chunk;
   }
   // Append new buffers and update size
@@ -368,6 +396,9 @@ Status BlockParser::DoParseSpecialized(const char* start, uint32_t size, bool is
     RETURN_NOT_OK(ParseChunk<SpecializedOptions>(&values_writer, &parsed_writer, data,
                                                  data_end, is_final, rows_in_chunk, &data,
                                                  &finished_parsing));
+    if (num_cols_ == -1) {
+      return ParseError("Empty CSV file or block: cannot infer number of columns");
+    }
   }
   while (!finished_parsing && data < data_end && num_rows_ < max_num_rows_) {
     // We know the number of columns, so can presize a values array for
diff --git a/cpp/src/arrow/csv/parser.h b/cpp/src/arrow/csv/parser.h
index 0e1a415..8a51574 100644
--- a/cpp/src/arrow/csv/parser.h
+++ b/cpp/src/arrow/csv/parser.h
@@ -67,7 +67,9 @@ class ARROW_EXPORT BlockParser {
   /// The last row may lack a trailing line separator.
   Status ParseFinal(const char* data, uint32_t size, uint32_t* out_size);
 
+  /// \brief Return the number of parsed rows
   int32_t num_rows() const { return num_rows_; }
+  /// \brief Return the number of parsed columns
   int32_t num_cols() const { return num_cols_; }
   /// \brief Return the total size in bytes of parsed data
   uint32_t num_bytes() const { return parsed_size_; }
diff --git a/cpp/src/arrow/csv/test-common.h b/cpp/src/arrow/csv/test-common.h
index dd0e0cf..ac73273 100644
--- a/cpp/src/arrow/csv/test-common.h
+++ b/cpp/src/arrow/csv/test-common.h
@@ -38,22 +38,30 @@ std::string MakeCSVData(std::vector<std::string> lines) {
 }
 
 // Make a BlockParser from a vector of lines representing a CSV file
-void MakeCSVParser(std::vector<std::string> lines, std::shared_ptr<BlockParser>* out) {
+void MakeCSVParser(std::vector<std::string> lines, ParseOptions options,
+                   std::shared_ptr<BlockParser>* out) {
   auto csv = MakeCSVData(lines);
-  auto parser = std::make_shared<BlockParser>(ParseOptions::Defaults());
+  auto parser = std::make_shared<BlockParser>(options);
   uint32_t out_size;
   ASSERT_OK(parser->Parse(csv.data(), static_cast<uint32_t>(csv.size()), &out_size));
   ASSERT_EQ(out_size, csv.size()) << "trailing CSV data not parsed";
   *out = parser;
 }
 
+void MakeCSVParser(std::vector<std::string> lines, std::shared_ptr<BlockParser>* out) {
+  MakeCSVParser(lines, ParseOptions::Defaults(), out);
+}
+
 // Make a BlockParser from a vector of strings representing a single CSV column
 void MakeColumnParser(std::vector<std::string> items, std::shared_ptr<BlockParser>* out) {
+  auto options = ParseOptions::Defaults();
+  // Need this to test for null (empty) values
+  options.ignore_empty_lines = false;
   std::vector<std::string> lines;
   for (const auto& item : items) {
     lines.push_back(item + '\n');
   }
-  MakeCSVParser(lines, out);
+  MakeCSVParser(lines, options, out);
   ASSERT_EQ((*out)->num_cols(), 1) << "Should have seen only 1 CSV column";
   ASSERT_EQ((*out)->num_rows(), items.size());
 }
diff --git a/python/pyarrow/_csv.pyx b/python/pyarrow/_csv.pyx
index e6488d0..5e98950 100644
--- a/python/pyarrow/_csv.pyx
+++ b/python/pyarrow/_csv.pyx
@@ -107,6 +107,10 @@ cdef class ParseOptions:
         Whether newline characters are allowed in CSV values.
         Setting this to True reduces the performance of multi-threaded
         CSV reading.
+    ignore_empty_lines: bool, optional (default True)
+        Whether empty lines are ignored in CSV input.
+        If False, an empty line is interpreted as containing a single empty
+        value (assuming a one-column CSV file).
     """
     cdef:
         CCSVParseOptions options
@@ -114,7 +118,8 @@ cdef class ParseOptions:
     __slots__ = ()
 
     def __init__(self, delimiter=None, quote_char=None, double_quote=None,
-                 escape_char=None, header_rows=None, newlines_in_values=None):
+                 escape_char=None, header_rows=None, newlines_in_values=None,
+                 ignore_empty_lines=None):
         self.options = CCSVParseOptions.Defaults()
         if delimiter is not None:
             self.delimiter = delimiter
@@ -128,6 +133,8 @@ cdef class ParseOptions:
             self.header_rows = header_rows
         if newlines_in_values is not None:
             self.newlines_in_values = newlines_in_values
+        if ignore_empty_lines is not None:
+            self.ignore_empty_lines = ignore_empty_lines
 
     @property
     def delimiter(self):
@@ -214,6 +221,19 @@ cdef class ParseOptions:
     def newlines_in_values(self, value):
         self.options.newlines_in_values = value
 
+    @property
+    def ignore_empty_lines(self):
+        """
+        Whether empty lines are ignored in CSV input.
+        If False, an empty line is interpreted as containing a single empty
+        value (assuming a one-column CSV file).
+        """
+        return self.options.ignore_empty_lines
+
+    @ignore_empty_lines.setter
+    def ignore_empty_lines(self, value):
+        self.options.ignore_empty_lines = value
+
 
 cdef class ConvertOptions:
     """
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index a3d356e..1320e00 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -942,6 +942,7 @@ cdef extern from "arrow/csv/api.h" namespace "arrow::csv" nogil:
         unsigned char escape_char
         int32_t header_rows
         c_bool newlines_in_values
+        c_bool ignore_empty_lines
 
         @staticmethod
         CCSVParseOptions Defaults()
diff --git a/python/pyarrow/tests/test_csv.py b/python/pyarrow/tests/test_csv.py
index c204fdf..46a77f6 100644
--- a/python/pyarrow/tests/test_csv.py
+++ b/python/pyarrow/tests/test_csv.py
@@ -84,6 +84,7 @@ def test_parse_options():
     assert opts.escape_char is False
     assert opts.header_rows == 1
     assert opts.newlines_in_values is False
+    assert opts.ignore_empty_lines is True
 
     opts.delimiter = 'x'
     assert opts.delimiter == 'x'
@@ -104,17 +105,22 @@ def test_parse_options():
     opts.newlines_in_values = True
     assert opts.newlines_in_values is True
 
+    opts.ignore_empty_lines = False
+    assert opts.ignore_empty_lines is False
+
     opts.header_rows = 2
     assert opts.header_rows == 2
 
     opts = cls(delimiter=';', quote_char='%', double_quote=False,
-               escape_char='\\', header_rows=2, newlines_in_values=True)
+               escape_char='\\', header_rows=2, newlines_in_values=True,
+               ignore_empty_lines=False)
     assert opts.delimiter == ';'
     assert opts.quote_char == '%'
     assert opts.double_quote is False
     assert opts.escape_char == '\\'
     assert opts.header_rows == 2
     assert opts.newlines_in_values is True
+    assert opts.ignore_empty_lines is False
 
 
 def test_convert_options():
@@ -206,9 +212,9 @@ class BaseTestCSVRead:
 
     def test_trivial(self):
         # A bit pointless, but at least it shouldn't crash
-        rows = b"\n\n"
+        rows = b",\n\n"
         table = self.read_bytes(rows)
-        assert table.to_pydict() == {'': [None]}
+        assert table.to_pydict() == {'': []}
 
     def test_invalid_csv(self):
         # Various CSV errors
@@ -218,9 +224,9 @@ class BaseTestCSVRead:
         rows = b"a,b,c\n1,2,3\n4"
         with pytest.raises(pa.ArrowInvalid, match="Expected 3 columns, got 1"):
             self.read_bytes(rows)
-        rows = b""
-        with pytest.raises(pa.ArrowInvalid, match="Empty CSV file"):
-            self.read_bytes(rows)
+        for rows in [b"", b"\n", b"\r\n", b"\r", b"\n\n"]:
+            with pytest.raises(pa.ArrowInvalid, match="Empty CSV file"):
+                self.read_bytes(rows)
 
     def test_options_delimiter(self):
         rows = b"a;b,c\nde,fg;eh\n"