You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "kevingurney (via GitHub)" <gi...@apache.org> on 2023/09/18 15:57:04 UTC

[GitHub] [arrow] kevingurney opened a new pull request, #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

kevingurney opened a new pull request, #37773:
URL: https://github.com/apache/arrow/pull/37773

   ### Rationale for this change
   
   To enable initial CSV I/O support, this PR adds `arrow.io.csv.TableReader` and `arrow.io.csv.TableWriter` MATLAB classes which work with `arrow.tabular.Table`s to the MATLAB interface.
   
   ### What changes are included in this PR?
   
   1. Added a new `arrow.io.csv.TableReader` class
   2. Added a new `arrow.io.csv.TableWriter` class
   
   **Example**
   ```matlab
   >> matlabTableWrite = array2table(rand(3))
   
   matlabTableWrite =
   
     3×3 table
   
        Var1        Var2       Var3  
       _______    ________    _______
   
       0.91131    0.091595    0.24594
       0.51315     0.27368    0.62119
       0.42942     0.88665    0.49501
   
   >> arrowTableWrite = arrow.table(matlabTableWrite)
   
   arrowTableWrite = 
   
   Var1: double
   Var2: double
   Var3: double
   ----
   Var1:
     [
       [
         0.9113083542736461,
         0.5131490075412158,
         0.42942202968065213
       ]
     ]
   Var2:
     [
       [
         0.09159480217154525,
         0.27367730380496647,
         0.8866478145458545
       ]
     ]
   Var3:
     [
       [
         0.2459443412735529,
         0.6211893868708748,
         0.49500739584280073
       ]
     ]
   
   >> writer = arrow.io.csv.TableWriter("example.csv")
   
   writer = 
   
     TableWriter with properties:
   
       Filename: "example.csv"
   
   >> writer.write(arrowTableWrite)
   
   >> reader = arrow.io.csv.TableReader("example.csv")
   
   reader = 
   
     TableReader with properties:
   
       Filename: "example.csv"
   
   >> arrowTableRead = reader.read()
   
   arrowTableRead = 
   
   Var1: double
   Var2: double
   Var3: double
   ----
   Var1:
     [
       [
         0.9113083542736461,
         0.5131490075412158,
         0.42942202968065213
       ]
     ]
   Var2:
     [
       [
         0.09159480217154525,
         0.27367730380496647,
         0.8866478145458545
       ]
     ]
   Var3:
     [
       [
         0.2459443412735529,
         0.6211893868708748,
         0.49500739584280073
       ]
     ]
   
   >> matlabTableRead = table(arrowTableRead)
   
   matlabTableRead =
   
     3×3 table
   
        Var1        Var2       Var3  
       _______    ________    _______
   
       0.91131    0.091595    0.24594
       0.51315     0.27368    0.62119
       0.42942     0.88665    0.49501
   
   >> isequal(arrowTableRead, arrowTableWrite)
   
   ans =
   
     logical
   
      1
   
   >> isequal(matlabTableRead, matlabTableWrite)
   
   ans =
   
     logical
   
      1
   ```
   
   ### Are these changes tested?
   
   Yes.
   
   1. Added new CSV I/O tests including `test/arrow/io/csv/tRoundTrip.m` and `test/arrow/io/csv/tError.m`.
   2. Both of these test classes inherit from a `CSVTest` superclass.
   
   ### Are there any user-facing changes?
   
   Yes.
   
   1. Users can now read and write CSV files using `arrow.io.csv.TableReader` and `arrow.io.csv.TableWriter`.
   
   ### Future Directions
   
   1. Expose [options](https://github.com/apache/arrow/blob/main/cpp/src/arrow/csv/options.h) for controlling CSV reading and writing in MATLAB.
   2. Add more read/write tests for null value handling and other datatypes beyond numeric and string values.
   4. Add a `RecordBatchReader` and `RecordBatchWriter` for CSV.
   5. Add support for more I/O formats like Parquet, JSON, ORC, Arrow IPC, etc.
   
   ### Notes
   
   1. Thank you @sgilmore10 for your help with this pull request!
   2. I chose to add both the `TableReader` and `TableWriter` in one pull request because it simplified testing. My apologies for the slightly lengthy pull request.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1727871858

   The failures for the `Dev / Source Release and Merge Script` CI workflow seem unrelated to this PR.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1724022938

   It looks like the `ARROW_CSV` component is currently not enabled in the Arrow C++ libraries `ExternalProject` build. I just pushed a commit that enables this. 


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kou commented on a diff in pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on code in PR #37773:
URL: https://github.com/apache/arrow/pull/37773#discussion_r1330880064


##########
matlab/src/cpp/arrow/matlab/io/csv/proxy/table_reader.cc:
##########
@@ -0,0 +1,93 @@
+// 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.
+
+#include "libmexclass/proxy/ProxyManager.h"
+
+#include "arrow/matlab/error/error.h"
+#include "arrow/matlab/io/csv/proxy/table_reader.h"
+#include "arrow/matlab/tabular/proxy/table.h"
+
+#include "arrow/util/utf8.h"
+
+#include "arrow/result.h"
+
+#include "arrow/io/file.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/csv/reader.h"
+#include "arrow/table.h"
+
+namespace arrow::matlab::io::csv::proxy {
+
+    TableReader::TableReader(const std::string& filename) : filename{filename} {
+        REGISTER_METHOD(TableReader, read);
+        REGISTER_METHOD(TableReader, getFilename);
+    }
+
+    libmexclass::proxy::MakeResult TableReader::make(const libmexclass::proxy::FunctionArguments& constructor_arguments) {
+        namespace mda = ::matlab::data;
+        using TableReaderProxy = arrow::matlab::io::csv::proxy::TableReader;
+
+        mda::StructArray args = constructor_arguments[0];
+        const mda::StringArray filename_utf16_mda = args[0]["Filename"];
+        const auto filename_utf16 = std::u16string(filename_utf16_mda[0]);
+        MATLAB_ASSIGN_OR_ERROR(const auto filename, arrow::util::UTF16StringToUTF8(filename_utf16), error::UNICODE_CONVERSION_ERROR_ID);
+
+        return std::make_shared<TableReaderProxy>(filename);
+    }
+
+    void TableReader::read(libmexclass::proxy::method::Context& context) {
+        namespace mda = ::matlab::data;
+        using namespace libmexclass::proxy;
+        namespace csv = ::arrow::csv;
+        using TableProxy = arrow::matlab::tabular::proxy::Table;
+
+        mda::ArrayFactory factory;
+
+        // Create a file input stream.
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(auto source, arrow::io::ReadableFile::Open(filename, arrow::default_memory_pool()), context, error::FAILED_TO_OPEN_FILE_FOR_READ);
+
+        const ::arrow::io::IOContext io_context;
+        const csv::ReadOptions read_options = csv::ReadOptions::Defaults();
+        const csv::ParseOptions parse_options = csv::ParseOptions::Defaults();
+        const csv::ConvertOptions convert_options = csv::ConvertOptions::Defaults();

Review Comment:
   Can we use `auto` here?



##########
matlab/src/cpp/arrow/matlab/io/csv/proxy/table_writer.cc:
##########
@@ -0,0 +1,86 @@
+// 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.
+
+#include "arrow/matlab/io/csv/proxy/table_writer.h"
+#include "arrow/matlab/tabular/proxy/table.h"
+#include "arrow/matlab/error/error.h"
+
+#include "arrow/result.h"
+#include "arrow/table.h"
+#include "arrow/util/utf8.h"
+
+#include "arrow/io/file.h"
+#include "arrow/csv/writer.h"
+#include "arrow/csv/options.h"
+
+#include "libmexclass/proxy/ProxyManager.h"
+
+namespace arrow::matlab::io::csv::proxy {
+
+    TableWriter::TableWriter(const std::string& filename) : filename{filename} {
+        REGISTER_METHOD(TableWriter, getFilename);
+        REGISTER_METHOD(TableWriter, write);
+    }
+
+    libmexclass::proxy::MakeResult TableWriter::make(const libmexclass::proxy::FunctionArguments& constructor_arguments) {
+        namespace mda = ::matlab::data;
+        mda::StructArray opts = constructor_arguments[0];
+        const mda::StringArray filename_mda = opts[0]["Filename"];
+        using TableWriterProxy = ::arrow::matlab::io::csv::proxy::TableWriter;
+
+        const auto filename_utf16 = std::u16string(filename_mda[0]);
+        MATLAB_ASSIGN_OR_ERROR(const auto filename_utf8,
+                               arrow::util::UTF16StringToUTF8(filename_utf16),
+                               error::UNICODE_CONVERSION_ERROR_ID);
+
+        return std::make_shared<TableWriterProxy>(filename_utf8);
+    }
+
+    void TableWriter::getFilename(libmexclass::proxy::method::Context& context) {
+        namespace mda = ::matlab::data;
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(const auto utf16_filename,
+                                            arrow::util::UTF8StringToUTF16(filename),
+                                            context,
+                                            error::UNICODE_CONVERSION_ERROR_ID);
+        mda::ArrayFactory factory;
+        auto str_mda = factory.createScalar(utf16_filename);
+        context.outputs[0] = str_mda;
+    }
+
+    void TableWriter::write(libmexclass::proxy::method::Context& context) {
+        namespace csv = ::arrow::csv;
+        namespace mda = ::matlab::data;
+        using TableProxy = ::arrow::matlab::tabular::proxy::Table;
+
+        mda::StructArray opts = context.inputs[0];
+        const mda::TypedArray<uint64_t> table_proxy_id_mda = opts[0]["TableProxyID"];
+        const uint64_t table_proxy_id = table_proxy_id_mda[0];
+
+        auto proxy = libmexclass::proxy::ProxyManager::getProxy(table_proxy_id);
+        auto table_proxy = std::static_pointer_cast<TableProxy>(proxy);
+        auto table = table_proxy->unwrap();
+
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(std::shared_ptr<arrow::io::OutputStream> output_stream,
+                                            arrow::io::FileOutputStream::Open(filename),
+                                            context,
+                                            error::FAILED_TO_OPEN_FILE_FOR_WRITE);
+        csv::WriteOptions options = csv::WriteOptions::Defaults();

Review Comment:
   Can we use `auto` here?



##########
matlab/src/cpp/arrow/matlab/io/csv/proxy/table_writer.cc:
##########
@@ -0,0 +1,86 @@
+// 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.
+
+#include "arrow/matlab/io/csv/proxy/table_writer.h"
+#include "arrow/matlab/tabular/proxy/table.h"
+#include "arrow/matlab/error/error.h"
+
+#include "arrow/result.h"
+#include "arrow/table.h"
+#include "arrow/util/utf8.h"
+
+#include "arrow/io/file.h"
+#include "arrow/csv/writer.h"
+#include "arrow/csv/options.h"
+
+#include "libmexclass/proxy/ProxyManager.h"
+
+namespace arrow::matlab::io::csv::proxy {
+
+    TableWriter::TableWriter(const std::string& filename) : filename{filename} {
+        REGISTER_METHOD(TableWriter, getFilename);
+        REGISTER_METHOD(TableWriter, write);
+    }
+
+    libmexclass::proxy::MakeResult TableWriter::make(const libmexclass::proxy::FunctionArguments& constructor_arguments) {
+        namespace mda = ::matlab::data;
+        mda::StructArray opts = constructor_arguments[0];
+        const mda::StringArray filename_mda = opts[0]["Filename"];
+        using TableWriterProxy = ::arrow::matlab::io::csv::proxy::TableWriter;
+
+        const auto filename_utf16 = std::u16string(filename_mda[0]);
+        MATLAB_ASSIGN_OR_ERROR(const auto filename_utf8,
+                               arrow::util::UTF16StringToUTF8(filename_utf16),
+                               error::UNICODE_CONVERSION_ERROR_ID);
+
+        return std::make_shared<TableWriterProxy>(filename_utf8);
+    }
+
+    void TableWriter::getFilename(libmexclass::proxy::method::Context& context) {
+        namespace mda = ::matlab::data;
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(const auto utf16_filename,
+                                            arrow::util::UTF8StringToUTF16(filename),
+                                            context,
+                                            error::UNICODE_CONVERSION_ERROR_ID);
+        mda::ArrayFactory factory;
+        auto str_mda = factory.createScalar(utf16_filename);
+        context.outputs[0] = str_mda;
+    }
+
+    void TableWriter::write(libmexclass::proxy::method::Context& context) {
+        namespace csv = ::arrow::csv;
+        namespace mda = ::matlab::data;
+        using TableProxy = ::arrow::matlab::tabular::proxy::Table;
+
+        mda::StructArray opts = context.inputs[0];
+        const mda::TypedArray<uint64_t> table_proxy_id_mda = opts[0]["TableProxyID"];
+        const uint64_t table_proxy_id = table_proxy_id_mda[0];
+
+        auto proxy = libmexclass::proxy::ProxyManager::getProxy(table_proxy_id);
+        auto table_proxy = std::static_pointer_cast<TableProxy>(proxy);
+        auto table = table_proxy->unwrap();
+
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(std::shared_ptr<arrow::io::OutputStream> output_stream,

Review Comment:
   Can we use `auto` 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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on a diff in pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on code in PR #37773:
URL: https://github.com/apache/arrow/pull/37773#discussion_r1331635242


##########
matlab/src/cpp/arrow/matlab/io/csv/proxy/table_writer.cc:
##########
@@ -0,0 +1,86 @@
+// 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.
+
+#include "arrow/matlab/io/csv/proxy/table_writer.h"
+#include "arrow/matlab/tabular/proxy/table.h"
+#include "arrow/matlab/error/error.h"
+
+#include "arrow/result.h"
+#include "arrow/table.h"
+#include "arrow/util/utf8.h"
+
+#include "arrow/io/file.h"
+#include "arrow/csv/writer.h"
+#include "arrow/csv/options.h"
+
+#include "libmexclass/proxy/ProxyManager.h"
+
+namespace arrow::matlab::io::csv::proxy {
+
+    TableWriter::TableWriter(const std::string& filename) : filename{filename} {
+        REGISTER_METHOD(TableWriter, getFilename);
+        REGISTER_METHOD(TableWriter, write);
+    }
+
+    libmexclass::proxy::MakeResult TableWriter::make(const libmexclass::proxy::FunctionArguments& constructor_arguments) {
+        namespace mda = ::matlab::data;
+        mda::StructArray opts = constructor_arguments[0];
+        const mda::StringArray filename_mda = opts[0]["Filename"];
+        using TableWriterProxy = ::arrow::matlab::io::csv::proxy::TableWriter;
+
+        const auto filename_utf16 = std::u16string(filename_mda[0]);
+        MATLAB_ASSIGN_OR_ERROR(const auto filename_utf8,
+                               arrow::util::UTF16StringToUTF8(filename_utf16),
+                               error::UNICODE_CONVERSION_ERROR_ID);
+
+        return std::make_shared<TableWriterProxy>(filename_utf8);
+    }
+
+    void TableWriter::getFilename(libmexclass::proxy::method::Context& context) {
+        namespace mda = ::matlab::data;
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(const auto utf16_filename,
+                                            arrow::util::UTF8StringToUTF16(filename),
+                                            context,
+                                            error::UNICODE_CONVERSION_ERROR_ID);
+        mda::ArrayFactory factory;
+        auto str_mda = factory.createScalar(utf16_filename);
+        context.outputs[0] = str_mda;
+    }
+
+    void TableWriter::write(libmexclass::proxy::method::Context& context) {
+        namespace csv = ::arrow::csv;
+        namespace mda = ::matlab::data;
+        using TableProxy = ::arrow::matlab::tabular::proxy::Table;
+
+        mda::StructArray opts = context.inputs[0];
+        const mda::TypedArray<uint64_t> table_proxy_id_mda = opts[0]["TableProxyID"];
+        const uint64_t table_proxy_id = table_proxy_id_mda[0];
+
+        auto proxy = libmexclass::proxy::ProxyManager::getProxy(table_proxy_id);
+        auto table_proxy = std::static_pointer_cast<TableProxy>(proxy);
+        auto table = table_proxy->unwrap();
+
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(std::shared_ptr<arrow::io::OutputStream> output_stream,
+                                            arrow::io::FileOutputStream::Open(filename),
+                                            context,
+                                            error::FAILED_TO_OPEN_FILE_FOR_WRITE);
+        csv::WriteOptions options = csv::WriteOptions::Defaults();

Review Comment:
   Fixed.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] conbench-apache-arrow[bot] commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "conbench-apache-arrow[bot] (via GitHub)" <gi...@apache.org>.
conbench-apache-arrow[bot] commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1728890093

   After merging your PR, Conbench analyzed the 5 benchmarking runs that have been run so far on merge-commit 2b34e37a956ac59b79e74da1dde8f037c9c88c5d.
   
   There were no benchmark performance regressions. 🎉
   
   The [full Conbench report](https://github.com/apache/arrow/runs/16990404803) has more details. It also includes information about possible false positives for unstable benchmarks that are known to sometimes produce them.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1724496660

   For reference - I've opened #37784 for removing `GoogleTest` support.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1728047186

   Unfortunately, the MATLAB CI workflows just started failing due to #37805.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1728101006

   I've opened #37808 to address the MATLAB CI failures.
   
   I'll merge this PR after #37808 has been merged.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1726437203

   **Update**: the code to remove `GoogleTest` support from the MATLAB interface was merged in #37784.
   
   Now that the Arrow C++ tests are no longer being built, enabling `ARROW_CSV` in the MATLAB build works without any issues.
   
   At this point, these changes should be ready to merge.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1728237480

   The MATLAB CI workflows are passing now.
   
   I will merge this PR.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] github-actions[bot] commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1723798102

   :warning: GitHub issue #37770 **has been automatically assigned in GitHub** to PR creator.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney merged pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney merged PR #37773:
URL: https://github.com/apache/arrow/pull/37773


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1727904787

   CI failures are related to #37803.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on a diff in pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on code in PR #37773:
URL: https://github.com/apache/arrow/pull/37773#discussion_r1331610978


##########
matlab/src/cpp/arrow/matlab/io/csv/proxy/table_writer.cc:
##########
@@ -0,0 +1,86 @@
+// 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.
+
+#include "arrow/matlab/io/csv/proxy/table_writer.h"
+#include "arrow/matlab/tabular/proxy/table.h"
+#include "arrow/matlab/error/error.h"
+
+#include "arrow/result.h"
+#include "arrow/table.h"
+#include "arrow/util/utf8.h"
+
+#include "arrow/io/file.h"
+#include "arrow/csv/writer.h"
+#include "arrow/csv/options.h"
+
+#include "libmexclass/proxy/ProxyManager.h"
+
+namespace arrow::matlab::io::csv::proxy {
+
+    TableWriter::TableWriter(const std::string& filename) : filename{filename} {
+        REGISTER_METHOD(TableWriter, getFilename);
+        REGISTER_METHOD(TableWriter, write);
+    }
+
+    libmexclass::proxy::MakeResult TableWriter::make(const libmexclass::proxy::FunctionArguments& constructor_arguments) {
+        namespace mda = ::matlab::data;
+        mda::StructArray opts = constructor_arguments[0];
+        const mda::StringArray filename_mda = opts[0]["Filename"];
+        using TableWriterProxy = ::arrow::matlab::io::csv::proxy::TableWriter;
+
+        const auto filename_utf16 = std::u16string(filename_mda[0]);
+        MATLAB_ASSIGN_OR_ERROR(const auto filename_utf8,
+                               arrow::util::UTF16StringToUTF8(filename_utf16),
+                               error::UNICODE_CONVERSION_ERROR_ID);
+
+        return std::make_shared<TableWriterProxy>(filename_utf8);
+    }
+
+    void TableWriter::getFilename(libmexclass::proxy::method::Context& context) {
+        namespace mda = ::matlab::data;
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(const auto utf16_filename,
+                                            arrow::util::UTF8StringToUTF16(filename),
+                                            context,
+                                            error::UNICODE_CONVERSION_ERROR_ID);
+        mda::ArrayFactory factory;
+        auto str_mda = factory.createScalar(utf16_filename);
+        context.outputs[0] = str_mda;
+    }
+
+    void TableWriter::write(libmexclass::proxy::method::Context& context) {
+        namespace csv = ::arrow::csv;
+        namespace mda = ::matlab::data;
+        using TableProxy = ::arrow::matlab::tabular::proxy::Table;
+
+        mda::StructArray opts = context.inputs[0];
+        const mda::TypedArray<uint64_t> table_proxy_id_mda = opts[0]["TableProxyID"];
+        const uint64_t table_proxy_id = table_proxy_id_mda[0];
+
+        auto proxy = libmexclass::proxy::ProxyManager::getProxy(table_proxy_id);
+        auto table_proxy = std::static_pointer_cast<TableProxy>(proxy);
+        auto table = table_proxy->unwrap();
+
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(std::shared_ptr<arrow::io::OutputStream> output_stream,

Review Comment:
   Fixed.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1724207859

   Enabling building of the `ARROW_CSV` component caused some build failures due to `RapidJSON` being inaccessible when building integration tests:
   
   https://github.com/apache/arrow/actions/runs/6225733757/job/16896790846?pr=37773#step:9:681.
   
   We could try to fix these build failures by explicitly installing `RapidJSON` into the GitHub Actions environments. However, considering that we were [already considering removing `GoogleTest` support from the MATLAB build system](#37532) - it might be better to just use this opportunity to do that (this is what the [Python bindings did, as well](https://github.com/apache/arrow/pull/14117)). 
   
   I have already confirmed that not building the Arrow C++ library tests when `ARROW_CSV` is enabled does work on all platforms in another CI job in `mathworks/arrow`:
   
   https://github.com/mathworks/arrow/actions/runs/6226385022
   
   I'll start working on a separate PR to remove `GoogleTest` support from the CMake build system for the MATLAB interface. In the meantime, I will leave this PR open, and I will rebase once `GoogleTest` support has been removed.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1727907999

   +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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1724284828

   For reference, I am working on the changes required to remove `GoogleTest` support from the CMake build system of the MATLAB interface here: https://github.com/mathworks/arrow/tree/GH-37532.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on a diff in pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on code in PR #37773:
URL: https://github.com/apache/arrow/pull/37773#discussion_r1331609136


##########
matlab/src/cpp/arrow/matlab/io/csv/proxy/table_reader.cc:
##########
@@ -0,0 +1,93 @@
+// 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.
+
+#include "libmexclass/proxy/ProxyManager.h"
+
+#include "arrow/matlab/error/error.h"
+#include "arrow/matlab/io/csv/proxy/table_reader.h"
+#include "arrow/matlab/tabular/proxy/table.h"
+
+#include "arrow/util/utf8.h"
+
+#include "arrow/result.h"
+
+#include "arrow/io/file.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/csv/reader.h"
+#include "arrow/table.h"
+
+namespace arrow::matlab::io::csv::proxy {
+
+    TableReader::TableReader(const std::string& filename) : filename{filename} {
+        REGISTER_METHOD(TableReader, read);
+        REGISTER_METHOD(TableReader, getFilename);
+    }
+
+    libmexclass::proxy::MakeResult TableReader::make(const libmexclass::proxy::FunctionArguments& constructor_arguments) {
+        namespace mda = ::matlab::data;
+        using TableReaderProxy = arrow::matlab::io::csv::proxy::TableReader;
+
+        mda::StructArray args = constructor_arguments[0];
+        const mda::StringArray filename_utf16_mda = args[0]["Filename"];
+        const auto filename_utf16 = std::u16string(filename_utf16_mda[0]);
+        MATLAB_ASSIGN_OR_ERROR(const auto filename, arrow::util::UTF16StringToUTF8(filename_utf16), error::UNICODE_CONVERSION_ERROR_ID);
+
+        return std::make_shared<TableReaderProxy>(filename);
+    }
+
+    void TableReader::read(libmexclass::proxy::method::Context& context) {
+        namespace mda = ::matlab::data;
+        using namespace libmexclass::proxy;
+        namespace csv = ::arrow::csv;
+        using TableProxy = arrow::matlab::tabular::proxy::Table;
+
+        mda::ArrayFactory factory;
+
+        // Create a file input stream.
+        MATLAB_ASSIGN_OR_ERROR_WITH_CONTEXT(auto source, arrow::io::ReadableFile::Open(filename, arrow::default_memory_pool()), context, error::FAILED_TO_OPEN_FILE_FOR_READ);
+
+        const ::arrow::io::IOContext io_context;
+        const csv::ReadOptions read_options = csv::ReadOptions::Defaults();
+        const csv::ParseOptions parse_options = csv::ParseOptions::Defaults();
+        const csv::ConvertOptions convert_options = csv::ConvertOptions::Defaults();

Review Comment:
   Fixed.



-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1728220467

   https://github.com/apache/arrow/pull/37808 has been merged.
   
   I just rebased these changes on top of `main` so that the latest MATLAB CI workflows will run.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kevingurney commented on pull request #37773: GH-37770: [MATLAB] Add CSV `TableReader` and `TableWriter` MATLAB classes

Posted by "kevingurney (via GitHub)" <gi...@apache.org>.
kevingurney commented on PR #37773:
URL: https://github.com/apache/arrow/pull/37773#issuecomment-1728237729

   +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: github-unsubscribe@arrow.apache.org

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