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/12 19:28:11 UTC

[GitHub] [arrow] sgilmore10 opened a new pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

sgilmore10 opened a new pull request #10305:
URL: https://github.com/apache/arrow/pull/10305


   **Overview**
   * The MEX functions ``featherreadmex`` and ``featherwritemex`` fail to build against the latest Arrow C++ APIs. These changes allow them to successfully build.
   * These changes require CMake version 3.20 or later in order to access the latest functionality exposed by [FindMatlab.cmake](https://cmake.org/cmake/help/latest/module/FindMatlab.html). We noticed that some Arrow project components, such as [Gandiva](https://arrow.apache.org/docs/developers/cpp/building.html?highlight=gandiva#cmake-version-requirements), require newer versions of CMake than the core Arrow C++ libraries.  If version 3.20 is too new, we're happy to find an alternative.
   * We couldn't find a way to read and write a table description for feather V1 files using the latest APIs. It looks like support for reading and writing descriptions was modified in pull request https://github.com/apache/arrow/pull/6694. For now, we've removed support for table descriptions.
   
   **Testing**
   * Built ``featherreadmex`` and ``featherwritemex`` on Windows 10 with Visual Studio 2019
   * Built ``featherreadmex`` and ``featherwritemex`` on macOS Big Sur (11.2.3) with GNU Make 3.81
   * Built ``featherreadmex`` and ``featherwritemex`` on Debian 10 with GNU Make GNU 4.2.1
   * Ran all tests in ``tfeather`` and ``tfeathermex`` on all platforms in MATLAB R2021a
   
   **Future Directions**
   * We did not detect the build failures due to the lack of CI integration. We hope to add CI support soon and will follow up with a mailing list discussion to talk through the details. 
   * These changes are temporary to allow us to have a clean slate to start developing the  [MATLAB Interface to Apache Arrow](https://github.com/apache/arrow/blob/master/matlab/doc/matlab_interface_for_apache_arrow_design.md).
   * Eventually we would like to support the full ranges of data types for feather V1 and feather V2.
   * In order to modernize the code, we plan to migrate to the [C++ MEX](https://www.mathworks.com/help/matlab/cpp-mex-file-applications.html) and [MATLAB Data Array](https://www.mathworks.com/help/matlab/matlab-data-array.html) APIs.
   * We are going to follow up with another pull request to update the README.md to provide more detailed platform-specific development instructions. 
   * The MATLAB based build system inside of the ``build_support`` folder is out of date.  We are not sure if we want to maintain a separate MATLAB based build system along side the CMake based one. We will follow up on this in the future via the mailing list or Jira. 
   
   We acknowledge there is a lot of information in this pull request. In the future, we will work in smaller increments. We felt a bigger pull request was necessary to get back to a working state.
   
   Thanks,
   Sarah


-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,32 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<io::RandomAccessFile> random_access_file,
+                        maybe_readable_file);
 
   // Open the Feather file for reading with a TableReader.
-  RETURN_NOT_OK(ipc::feather::TableReader::Open(random_access_file,
-                                                &(*feather_reader)->table_reader_));
-
-  // Read the table metadata from the Feather file.
-  (*feather_reader)->num_rows_ = (*feather_reader)->table_reader_->num_rows();
-  (*feather_reader)->num_variables_ = (*feather_reader)->table_reader_->num_columns();
-  (*feather_reader)->description_ =
-      (*feather_reader)->table_reader_->HasDescription()
-          ? (*feather_reader)->table_reader_->GetDescription()
-          : "";
-
-  if ((*feather_reader)->num_rows_ > internal::MAX_MATLAB_SIZE ||
-      (*feather_reader)->num_variables_ > internal::MAX_MATLAB_SIZE) {
-    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
-                      "The table size exceeds MATLAB limits: %u x %u",
-                      (*feather_reader)->num_rows_, (*feather_reader)->num_variables_);
+  arrow::Result<std::shared_ptr<ipc::feather::Reader>> maybe_reader =
+      ipc::feather::Reader::Open(random_access_file);
+  ARROW_ASSIGN_OR_RAISE(auto reader, maybe_reader);

Review comment:
       I think we still need auto there. reader's a new variable I've never declared above.




-- 
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 change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,32 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<io::RandomAccessFile> random_access_file,
+                        maybe_readable_file);
 
   // Open the Feather file for reading with a TableReader.
-  RETURN_NOT_OK(ipc::feather::TableReader::Open(random_access_file,
-                                                &(*feather_reader)->table_reader_));
-
-  // Read the table metadata from the Feather file.
-  (*feather_reader)->num_rows_ = (*feather_reader)->table_reader_->num_rows();
-  (*feather_reader)->num_variables_ = (*feather_reader)->table_reader_->num_columns();
-  (*feather_reader)->description_ =
-      (*feather_reader)->table_reader_->HasDescription()
-          ? (*feather_reader)->table_reader_->GetDescription()
-          : "";
-
-  if ((*feather_reader)->num_rows_ > internal::MAX_MATLAB_SIZE ||
-      (*feather_reader)->num_variables_ > internal::MAX_MATLAB_SIZE) {
-    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
-                      "The table size exceeds MATLAB limits: %u x %u",
-                      (*feather_reader)->num_rows_, (*feather_reader)->num_variables_);
+  arrow::Result<std::shared_ptr<ipc::feather::Reader>> maybe_reader =
+      ipc::feather::Reader::Open(random_access_file);
+  ARROW_ASSIGN_OR_RAISE(auto reader, maybe_reader);

Review comment:
       Oh, sorry. You're right.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_writer.cc
##########
@@ -26,18 +24,53 @@
 #include <arrow/status.h>
 #include <arrow/table.h>
 #include <arrow/type.h>
-#include <arrow/util/bit-util.h>
-
+#include <arrow/util/bit_util.h>
+#include <arrow/util/bitmap_generate.h>
+#include <arrow/util/key_value_metadata.h>
 #include <mex.h>
 
-#include "feather_writer.h"
+#include <cmath>
+#include <functional> /* for std::multiplies */
+#include <numeric>    /* for std::accumulate */

Review comment:
       good to know. I'll update 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.

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



[GitHub] [arrow] kou commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,32 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);

Review comment:
       ```suggestion
     ARROW_ASSIGN_OR_RAISE(auto readable_file,  io::ReadableFile::Open(filename));
   ```

##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,32 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<io::RandomAccessFile> random_access_file,
+                        maybe_readable_file);

Review comment:
       Do we need this cast?
   
   I think that `arrow::ipc::feather::Reader::Open()` can accept `arrow::io::ReadableFile` because `arrow::io::ReadableFile` is a subclass of `arrow::io::RandomAccessFile`.

##########
File path: matlab/src/feather_reader.cc
##########
@@ -52,11 +57,12 @@ mxArray* ReadNumericVariableData(const std::shared_ptr<Array>& column) {
   mxArray* variable_data =
       mxCreateNumericMatrix(column->length(), 1, matlab_class_id, mxREAL);
 
-  std::shared_ptr<ArrowArrayType> integer_array =
+  std::shared_ptr<ArrowArrayType> arrow_numeric_array =
       std::static_pointer_cast<ArrowArrayType>(column);
 
   // Get a raw pointer to the Arrow array data.
-  const MatlabType* source = integer_array->raw_values();
+  const MatlabType* source =
+      reinterpret_cast<const MatlabType*>(arrow_numeric_array->values()->data());

Review comment:
       Does this work with sliced array?
   
   What is the problem of `raw_values()`?

##########
File path: matlab/src/feather_reader.cc
##########
@@ -52,11 +57,12 @@ mxArray* ReadNumericVariableData(const std::shared_ptr<Array>& column) {
   mxArray* variable_data =
       mxCreateNumericMatrix(column->length(), 1, matlab_class_id, mxREAL);
 
-  std::shared_ptr<ArrowArrayType> integer_array =
+  std::shared_ptr<ArrowArrayType> arrow_numeric_array =

Review comment:
       ```suggestion
     auto arrow_numeric_array =
   ```

##########
File path: matlab/src/feather_writer.cc
##########
@@ -248,60 +279,42 @@ Status FeatherWriter::Open(const std::string& filename,
   *feather_writer = std::shared_ptr<FeatherWriter>(new FeatherWriter());
 
   // Open a FileOutputStream corresponding to the provided filename.
-  std::shared_ptr<io::OutputStream> writable_file(nullptr);
-  ARROW_RETURN_NOT_OK(io::FileOutputStream::Open(filename, &writable_file));
-
-  // TableWriter::Open expects a shared_ptr to an OutputStream.
-  // Open the Feather file for writing with a TableWriter.
-  return ipc::feather::TableWriter::Open(writable_file,
-                                         &(*feather_writer)->table_writer_);
-}
-
-// Write table metadata to the Feather file from a mxArray*.
-void FeatherWriter::WriteMetadata(const mxArray* metadata) {
-  // Verify that all required fieldnames are provided.
-  internal::ValidateMxStructField(metadata, "Description", mxCHAR_CLASS, true);
-  internal::ValidateMxStructField(metadata, "NumRows", mxDOUBLE_CLASS, false);
-  internal::ValidateMxStructField(metadata, "NumVariables", mxDOUBLE_CLASS, false);
-
-  // Convert Description to a std::string and set on FeatherWriter and TableWriter.
-  std::string description =
-      internal::MxArrayToString(mxGetField(metadata, 0, "Description"));
-  this->description_ = description;
-  this->table_writer_->SetDescription(description);
-
-  // Get the NumRows field in the struct array and set on TableWriter.
-  this->num_rows_ = static_cast<int64_t>(mxGetScalar(mxGetField(metadata, 0, "NumRows")));
-  this->table_writer_->SetNumRows(this->num_rows_);
-
-  // Get the total number of variables. This is checked later for consistency with
-  // the provided number of columns before finishing the file write.
-  this->num_variables_ =
-      static_cast<int64_t>(mxGetScalar(mxGetField(metadata, 0, "NumVariables")));
+  arrow::Result<std::shared_ptr<arrow::io::OutputStream>> maybe_file_output_stream =
+      io::FileOutputStream::Open(filename, &((*feather_writer)->file_output_stream_));
+  ARROW_ASSIGN_OR_RAISE((*feather_writer)->file_output_stream_, maybe_file_output_stream);

Review comment:
       ```suggestion
     ARROW_ASSIGN_OR_RAISE((*feather_writer)->file_output_stream_,
         io::FileOutputStream::Open(filename, &((*feather_writer)->file_output_stream_));
   ```

##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,32 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<io::RandomAccessFile> random_access_file,
+                        maybe_readable_file);
 
   // Open the Feather file for reading with a TableReader.
-  RETURN_NOT_OK(ipc::feather::TableReader::Open(random_access_file,
-                                                &(*feather_reader)->table_reader_));
-
-  // Read the table metadata from the Feather file.
-  (*feather_reader)->num_rows_ = (*feather_reader)->table_reader_->num_rows();
-  (*feather_reader)->num_variables_ = (*feather_reader)->table_reader_->num_columns();
-  (*feather_reader)->description_ =
-      (*feather_reader)->table_reader_->HasDescription()
-          ? (*feather_reader)->table_reader_->GetDescription()
-          : "";
-
-  if ((*feather_reader)->num_rows_ > internal::MAX_MATLAB_SIZE ||
-      (*feather_reader)->num_variables_ > internal::MAX_MATLAB_SIZE) {
-    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
-                      "The table size exceeds MATLAB limits: %u x %u",
-                      (*feather_reader)->num_rows_, (*feather_reader)->num_variables_);
+  arrow::Result<std::shared_ptr<ipc::feather::Reader>> maybe_reader =
+      ipc::feather::Reader::Open(random_access_file);
+  ARROW_ASSIGN_OR_RAISE(auto reader, maybe_reader);
+
+  // Set the internal reader_ object.
+  (*feather_reader)->reader_ = reader;
+
+  // Check the feather file version
+  int version = reader->version();
+  if (version == ipc::feather::kFeatherV2Version) {
+    return Status::NotImplemented("Support for Feather V2 has not been implemented.");
+  } else if (version != ipc::feather::kFeatherV1Version) {
+    return Status::Invalid("Unknown Feather format version.");
   }
 
+  // read the table metadata from the Feather file
+  std::shared_ptr<Schema> schema = reader->schema();
+  (*feather_reader)->num_variables_ = schema->num_fields();

Review comment:
       ```suggestion
     (*feather_reader)->num_variables_ = reader->schema()->num_fields();
   ```

##########
File path: matlab/src/feather_reader.cc
##########
@@ -242,16 +243,34 @@ mxArray* FeatherReader::ReadVariables() const {
   mxArray* variables =
       mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
 
-  // Read all the table variables in the Feather file into memory.
+  std::shared_ptr<arrow::Table> table = nullptr;
+  arrow::Status status = reader_->Read(&table);
+  if (!status.ok()) {
+    std::string err_msg =
+        "Failed to read arrow::Table from Feather file. Reason: " + status.message();
+    mexErrMsgIdAndTxt("MATLAB:arrow:FeatherReader::FailedToReadTable", err_msg.c_str());
+  }
+
+  // Set the number of rows
+  num_rows_ = table->num_rows();
+
+  if (num_rows_ > internal::MAX_MATLAB_SIZE ||
+      num_variables_ > internal::MAX_MATLAB_SIZE) {
+    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
+                      "The table size exceeds MATLAB limits: %u x %u", num_rows_,
+                      num_variables_);
+  }
+
+  std::vector<std::string> column_names = table->ColumnNames();
+
   for (int64_t i = 0; i < num_variables_; ++i) {
-    std::shared_ptr<ChunkedArray> column;
-    util::HandleStatus(table_reader_->GetColumn(i, &column));
+    std::shared_ptr<ChunkedArray> column = table->column(i);

Review comment:
       ```suggestion
       auto column = table->column(i);
   ```

##########
File path: matlab/src/feather_reader.cc
##########
@@ -242,16 +243,34 @@ mxArray* FeatherReader::ReadVariables() const {
   mxArray* variables =
       mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
 
-  // Read all the table variables in the Feather file into memory.
+  std::shared_ptr<arrow::Table> table = nullptr;
+  arrow::Status status = reader_->Read(&table);
+  if (!status.ok()) {
+    std::string err_msg =
+        "Failed to read arrow::Table from Feather file. Reason: " + status.message();
+    mexErrMsgIdAndTxt("MATLAB:arrow:FeatherReader::FailedToReadTable", err_msg.c_str());
+  }
+
+  // Set the number of rows
+  num_rows_ = table->num_rows();
+
+  if (num_rows_ > internal::MAX_MATLAB_SIZE ||
+      num_variables_ > internal::MAX_MATLAB_SIZE) {
+    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
+                      "The table size exceeds MATLAB limits: %u x %u", num_rows_,
+                      num_variables_);
+  }
+
+  std::vector<std::string> column_names = table->ColumnNames();

Review comment:
       ```suggestion
     auto column_names = table->ColumnNames();
   ```

##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,32 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<io::RandomAccessFile> random_access_file,
+                        maybe_readable_file);
 
   // Open the Feather file for reading with a TableReader.
-  RETURN_NOT_OK(ipc::feather::TableReader::Open(random_access_file,
-                                                &(*feather_reader)->table_reader_));
-
-  // Read the table metadata from the Feather file.
-  (*feather_reader)->num_rows_ = (*feather_reader)->table_reader_->num_rows();
-  (*feather_reader)->num_variables_ = (*feather_reader)->table_reader_->num_columns();
-  (*feather_reader)->description_ =
-      (*feather_reader)->table_reader_->HasDescription()
-          ? (*feather_reader)->table_reader_->GetDescription()
-          : "";
-
-  if ((*feather_reader)->num_rows_ > internal::MAX_MATLAB_SIZE ||
-      (*feather_reader)->num_variables_ > internal::MAX_MATLAB_SIZE) {
-    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
-                      "The table size exceeds MATLAB limits: %u x %u",
-                      (*feather_reader)->num_rows_, (*feather_reader)->num_variables_);
+  arrow::Result<std::shared_ptr<ipc::feather::Reader>> maybe_reader =
+      ipc::feather::Reader::Open(random_access_file);
+  ARROW_ASSIGN_OR_RAISE(auto reader, maybe_reader);

Review comment:
       ```suggestion
     ARROW_ASSIGN_OR_RAISE(reader, ipc::feather::Reader::Open(readable_file))
   ```

##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +329,43 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);

Review comment:
       ```suggestion
       auto datatype = internal::ConvertMatlabTypeStringToArrowDataType(type_str);
   ```

##########
File path: matlab/src/feather_reader.cc
##########
@@ -242,16 +243,34 @@ mxArray* FeatherReader::ReadVariables() const {
   mxArray* variables =
       mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
 
-  // Read all the table variables in the Feather file into memory.
+  std::shared_ptr<arrow::Table> table = nullptr;
+  arrow::Status status = reader_->Read(&table);

Review comment:
       ```suggestion
     auto status = reader_->Read(&table);
   ```

##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,32 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<io::RandomAccessFile> random_access_file,
+                        maybe_readable_file);
 
   // Open the Feather file for reading with a TableReader.
-  RETURN_NOT_OK(ipc::feather::TableReader::Open(random_access_file,
-                                                &(*feather_reader)->table_reader_));
-
-  // Read the table metadata from the Feather file.
-  (*feather_reader)->num_rows_ = (*feather_reader)->table_reader_->num_rows();
-  (*feather_reader)->num_variables_ = (*feather_reader)->table_reader_->num_columns();
-  (*feather_reader)->description_ =
-      (*feather_reader)->table_reader_->HasDescription()
-          ? (*feather_reader)->table_reader_->GetDescription()
-          : "";
-
-  if ((*feather_reader)->num_rows_ > internal::MAX_MATLAB_SIZE ||
-      (*feather_reader)->num_variables_ > internal::MAX_MATLAB_SIZE) {
-    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
-                      "The table size exceeds MATLAB limits: %u x %u",
-                      (*feather_reader)->num_rows_, (*feather_reader)->num_variables_);
+  arrow::Result<std::shared_ptr<ipc::feather::Reader>> maybe_reader =
+      ipc::feather::Reader::Open(random_access_file);
+  ARROW_ASSIGN_OR_RAISE(auto reader, maybe_reader);
+
+  // Set the internal reader_ object.
+  (*feather_reader)->reader_ = reader;
+
+  // Check the feather file version
+  int version = reader->version();

Review comment:
       ```suggestion
     auto version = reader->version();
   ```

##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +329,43 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);
+    std::shared_ptr<arrow::Field> field =
+        std::make_shared<arrow::Field>(name_str, datatype);
+
+    arrow::Result<std::shared_ptr<ResizableBuffer>> maybe_buffer =
+        arrow::AllocateResizableBuffer(internal::BitPackedLength(num_rows_));
+    ARROW_ASSIGN_OR_RAISE(auto validity_bitmap, maybe_buffer);

Review comment:
       ```suggestion
       ARROW_ASSIGN_OR_RAISE(auto validity_bitmap,
           arrow::AllocateResizableBuffer(internal::BitPackedLength(num_rows_));
   ```

##########
File path: matlab/src/feather_reader.cc
##########
@@ -242,16 +243,34 @@ mxArray* FeatherReader::ReadVariables() const {
   mxArray* variables =
       mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
 
-  // Read all the table variables in the Feather file into memory.
+  std::shared_ptr<arrow::Table> table = nullptr;
+  arrow::Status status = reader_->Read(&table);
+  if (!status.ok()) {
+    std::string err_msg =
+        "Failed to read arrow::Table from Feather file. Reason: " + status.message();
+    mexErrMsgIdAndTxt("MATLAB:arrow:FeatherReader::FailedToReadTable", err_msg.c_str());

Review comment:
       ```suggestion
       mexErrMsgIdAndTxt("MATLAB:arrow:FeatherReader::FailedToReadTable",
                         "Failed to read arrow::Table from Feather file. Reason: %s",
                         status.message());
   ```

##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +329,43 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);
+    std::shared_ptr<arrow::Field> field =
+        std::make_shared<arrow::Field>(name_str, datatype);
+
+    arrow::Result<std::shared_ptr<ResizableBuffer>> maybe_buffer =
+        arrow::AllocateResizableBuffer(internal::BitPackedLength(num_rows_));
+    ARROW_ASSIGN_OR_RAISE(auto validity_bitmap, maybe_buffer);
+
     // Populate bit-packed arrow::Buffer using validity data in the mxArray*.
     internal::BitPackBuffer(valid, validity_bitmap);
 
     // Wrap mxArray data in an arrow::Array of the equivalent type.
-    std::unique_ptr<Array> array =
+    std::shared_ptr<Array> array =

Review comment:
       ```suggestion
       auto array =
   ```

##########
File path: matlab/CMakeLists.txt
##########
@@ -29,22 +30,51 @@ if(EXISTS "${CPP_CMAKE_MODULES}")
   set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CPP_CMAKE_MODULES})
 endif()
 
-## Arrow is Required
+set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CMAKE_SOURCE_DIR}/cmake_modules)
+
+# Arrow is Required
 find_package(Arrow REQUIRED)
 
-## MATLAB is required to be installed to build MEX interfaces
-set(MATLAB_ADDITIONAL_VERSIONS "R2018a=9.4")
-find_package(Matlab REQUIRED MX_LIBRARY)
-
-# Build featherread mex file based on the arrow shared library
-matlab_add_mex(NAME featherreadmex
-               SRC src/featherreadmex.cc src/feather_reader.cc src/util/handle_status.cc
-                   src/util/unicode_conversion.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherreadmex PRIVATE ${ARROW_INCLUDE_DIR})
-
-# Build featherwrite mex file based on the arrow shared library
-matlab_add_mex(NAME featherwritemex
-               SRC src/featherwritemex.cc src/feather_writer.cc src/util/handle_status.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherwritemex PRIVATE ${ARROW_INCLUDE_DIR})
+# MATLAB is Required
+find_package(Matlab REQUIRED)
+
+# Construct the absolute path to featherread's source files
+set(featherread_sources featherreadmex.cc feather_reader.cc util/handle_status.cc
+                        util/unicode_conversion.cc)
+list(TRANSFORM featherread_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherreadmex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherreadmex
+               SRC
+               ${featherread_sources}
+               LINK_TO
+               arrow_shared)
+
+# Construct the absolute path to featherwrite's source files
+set(featherwrite_sources featherwritemex.cc feather_writer.cc util/handle_status.cc
+                         util/unicode_conversion.cc)
+list(TRANSFORM featherwrite_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherwritemex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherwritemex
+               SRC
+               ${featherwrite_sources}
+               LINK_TO
+               arrow_shared)
+
+# Ensure the MEX binaries are placed in the src directory on all platforms

Review comment:
       Thanks.
   I understand. ( https://github.com/apache/arrow/blob/master/matlab/test/tfeathermex.m#L26 is the code that adds the source directory to the MATLAB search path.)
   
   Generally, we should not change anything files in the source directory with out-of-source build. Can we also add the build directory to the MATLAB search path? Is it difficult?

##########
File path: matlab/src/feather_reader.cc
##########
@@ -242,16 +245,34 @@ mxArray* FeatherReader::ReadVariables() const {
   mxArray* variables =
       mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
 
-  // Read all the table variables in the Feather file into memory.
+  // Read the entire table in the Feather file into memory.
+  std::shared_ptr<arrow::Table> table = nullptr;

Review comment:
       ```suggestion
     std::shared_ptr<arrow::Table> table;
   ```

##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +329,43 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);
+    std::shared_ptr<arrow::Field> field =
+        std::make_shared<arrow::Field>(name_str, datatype);
+
+    arrow::Result<std::shared_ptr<ResizableBuffer>> maybe_buffer =
+        arrow::AllocateResizableBuffer(internal::BitPackedLength(num_rows_));
+    ARROW_ASSIGN_OR_RAISE(auto validity_bitmap, maybe_buffer);
+
     // Populate bit-packed arrow::Buffer using validity data in the mxArray*.
     internal::BitPackBuffer(valid, validity_bitmap);
 
     // Wrap mxArray data in an arrow::Array of the equivalent type.
-    std::unique_ptr<Array> array =
+    std::shared_ptr<Array> array =
         internal::WriteVariableData(data, type_str, validity_bitmap);
 
     // Verify that the arrow::Array has the right number of elements.
-    internal::ValidateNumRows(array->length(), this->num_rows_);
+    internal::ValidateNumRows(array->length(), num_rows_);
 
-    // Write another column to the Feather file.
-    ARROW_RETURN_NOT_OK(this->table_writer_->Append(name_str, *array));
+    // Append the field to the schema builder
+    RETURN_NOT_OK(schema_builder.AddField(field));
+
+    // Store the table column
+    table_columns.push_back(array);
   }
+  // Create the table schema
+  arrow::Result<std::shared_ptr<arrow::Schema>> maybe_table_schema =
+      schema_builder.Finish();
+  ARROW_ASSIGN_OR_RAISE(auto table_schema, maybe_table_schema);

Review comment:
       ```suggestion
     ARROW_ASSIGN_OR_RAISE(auto table_schema, schema_builder.Finish());
   ```

##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +329,43 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);
+    std::shared_ptr<arrow::Field> field =
+        std::make_shared<arrow::Field>(name_str, datatype);

Review comment:
       ```suggestion
       auto field = std::make_shared<arrow::Field>(name_str, datatype);
   ```




-- 
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 #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Sorry for the long delay - I kicked off the other CI pipelines here. 
   
   If you rebase, that'll let you re-run the ARM build - note it's been somewhat flaky but should have gotten better recently (we've switched to a newer machine type, IIRC). 
   
   Also, did you have someone in mind to review this? If not we can find someone.


-- 
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] kou commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/CMakeLists.txt
##########
@@ -29,22 +30,51 @@ if(EXISTS "${CPP_CMAKE_MODULES}")
   set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CPP_CMAKE_MODULES})
 endif()
 
-## Arrow is Required
+set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CMAKE_SOURCE_DIR}/cmake_modules)
+
+# Arrow is Required
 find_package(Arrow REQUIRED)
 
-## MATLAB is required to be installed to build MEX interfaces
-set(MATLAB_ADDITIONAL_VERSIONS "R2018a=9.4")
-find_package(Matlab REQUIRED MX_LIBRARY)
-
-# Build featherread mex file based on the arrow shared library
-matlab_add_mex(NAME featherreadmex
-               SRC src/featherreadmex.cc src/feather_reader.cc src/util/handle_status.cc
-                   src/util/unicode_conversion.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherreadmex PRIVATE ${ARROW_INCLUDE_DIR})
-
-# Build featherwrite mex file based on the arrow shared library
-matlab_add_mex(NAME featherwritemex
-               SRC src/featherwritemex.cc src/feather_writer.cc src/util/handle_status.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherwritemex PRIVATE ${ARROW_INCLUDE_DIR})
+# MATLAB is Required
+find_package(Matlab REQUIRED)
+
+# Construct the absolute path to featherread's source files
+set(featherread_sources featherreadmex.cc feather_reader.cc util/handle_status.cc
+                        util/unicode_conversion.cc)
+list(TRANSFORM featherread_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherreadmex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherreadmex
+               SRC
+               ${featherread_sources}
+               LINK_TO
+               arrow_shared)
+
+# Construct the absolute path to featherwrite's source files
+set(featherwrite_sources featherwritemex.cc feather_writer.cc util/handle_status.cc
+                         util/unicode_conversion.cc)
+list(TRANSFORM featherwrite_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherwritemex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherwritemex
+               SRC
+               ${featherwrite_sources}
+               LINK_TO
+               arrow_shared)
+
+# Ensure the MEX binaries are placed in the src directory on all platforms

Review comment:
       I was just wondering. Why do we need to place the binaries in the source directory?




-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/CMakeLists.txt
##########
@@ -29,22 +30,51 @@ if(EXISTS "${CPP_CMAKE_MODULES}")
   set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CPP_CMAKE_MODULES})
 endif()
 
-## Arrow is Required
+set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CMAKE_SOURCE_DIR}/cmake_modules)
+
+# Arrow is Required
 find_package(Arrow REQUIRED)
 
-## MATLAB is required to be installed to build MEX interfaces
-set(MATLAB_ADDITIONAL_VERSIONS "R2018a=9.4")
-find_package(Matlab REQUIRED MX_LIBRARY)
-
-# Build featherread mex file based on the arrow shared library
-matlab_add_mex(NAME featherreadmex
-               SRC src/featherreadmex.cc src/feather_reader.cc src/util/handle_status.cc
-                   src/util/unicode_conversion.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherreadmex PRIVATE ${ARROW_INCLUDE_DIR})
-
-# Build featherwrite mex file based on the arrow shared library
-matlab_add_mex(NAME featherwritemex
-               SRC src/featherwritemex.cc src/feather_writer.cc src/util/handle_status.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherwritemex PRIVATE ${ARROW_INCLUDE_DIR})
+# MATLAB is Required
+find_package(Matlab REQUIRED)
+
+# Construct the absolute path to featherread's source files
+set(featherread_sources featherreadmex.cc feather_reader.cc util/handle_status.cc
+                        util/unicode_conversion.cc)
+list(TRANSFORM featherread_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherreadmex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherreadmex
+               SRC
+               ${featherread_sources}
+               LINK_TO
+               arrow_shared)
+
+# Construct the absolute path to featherwrite's source files
+set(featherwrite_sources featherwritemex.cc feather_writer.cc util/handle_status.cc
+                         util/unicode_conversion.cc)
+list(TRANSFORM featherwrite_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherwritemex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherwritemex
+               SRC
+               ${featherwrite_sources}
+               LINK_TO
+               arrow_shared)
+
+# Ensure the MEX binaries are placed in the src directory on all platforms

Review comment:
       Yeah, we can definitely investigate both approaches and see which one is preferable. I created a [jira task](https://issues.apache.org/jira/browse/ARROW-13185) to look into this in a future 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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +331,46 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);
+    std::shared_ptr<arrow::Field> field =
+        std::make_shared<arrow::Field>(name_str, datatype);
+
+    arrow::Result<std::shared_ptr<ResizableBuffer>> maybe_buffer =
+        arrow::AllocateResizableBuffer(internal::BitPackedLength(num_rows_));
+    RETURN_NOT_OK(maybe_buffer);
+    std::shared_ptr<ResizableBuffer> validity_bitmap = maybe_buffer.ValueOrDie();
+
     // Populate bit-packed arrow::Buffer using validity data in the mxArray*.
     internal::BitPackBuffer(valid, validity_bitmap);
 
     // Wrap mxArray data in an arrow::Array of the equivalent type.
-    std::unique_ptr<Array> array =
+    std::shared_ptr<Array> array =
         internal::WriteVariableData(data, type_str, validity_bitmap);
 
     // Verify that the arrow::Array has the right number of elements.
-    internal::ValidateNumRows(array->length(), this->num_rows_);
+    internal::ValidateNumRows(array->length(), num_rows_);
+
+    // Append the field to the schema builder
+    RETURN_NOT_OK(schema_builder.AddField(field));
 
-    // Write another column to the Feather file.
-    ARROW_RETURN_NOT_OK(this->table_writer_->Append(name_str, *array));
+    // Store the table column
+    table_columns.push_back(array);
   }
+  // Create the table schema
+  arrow::Result<std::shared_ptr<arrow::Schema>> table_schema_result =
+      schema_builder.Finish();
+  RETURN_NOT_OK(table_schema_result);
+
+  std::shared_ptr<arrow::Schema> table_schema = table_schema_result.ValueOrDie();

Review comment:
       will do.

##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +331,46 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);
+    std::shared_ptr<arrow::Field> field =
+        std::make_shared<arrow::Field>(name_str, datatype);
+
+    arrow::Result<std::shared_ptr<ResizableBuffer>> maybe_buffer =
+        arrow::AllocateResizableBuffer(internal::BitPackedLength(num_rows_));
+    RETURN_NOT_OK(maybe_buffer);
+    std::shared_ptr<ResizableBuffer> validity_bitmap = maybe_buffer.ValueOrDie();

Review comment:
       will do.

##########
File path: matlab/src/feather_writer.cc
##########
@@ -248,60 +279,44 @@ Status FeatherWriter::Open(const std::string& filename,
   *feather_writer = std::shared_ptr<FeatherWriter>(new FeatherWriter());
 
   // Open a FileOutputStream corresponding to the provided filename.
-  std::shared_ptr<io::OutputStream> writable_file(nullptr);
-  ARROW_RETURN_NOT_OK(io::FileOutputStream::Open(filename, &writable_file));
-
-  // TableWriter::Open expects a shared_ptr to an OutputStream.
-  // Open the Feather file for writing with a TableWriter.
-  return ipc::feather::TableWriter::Open(writable_file,
-                                         &(*feather_writer)->table_writer_);
-}
-
-// Write table metadata to the Feather file from a mxArray*.
-void FeatherWriter::WriteMetadata(const mxArray* metadata) {
-  // Verify that all required fieldnames are provided.
-  internal::ValidateMxStructField(metadata, "Description", mxCHAR_CLASS, true);
-  internal::ValidateMxStructField(metadata, "NumRows", mxDOUBLE_CLASS, false);
-  internal::ValidateMxStructField(metadata, "NumVariables", mxDOUBLE_CLASS, false);
-
-  // Convert Description to a std::string and set on FeatherWriter and TableWriter.
-  std::string description =
-      internal::MxArrayToString(mxGetField(metadata, 0, "Description"));
-  this->description_ = description;
-  this->table_writer_->SetDescription(description);
-
-  // Get the NumRows field in the struct array and set on TableWriter.
-  this->num_rows_ = static_cast<int64_t>(mxGetScalar(mxGetField(metadata, 0, "NumRows")));
-  this->table_writer_->SetNumRows(this->num_rows_);
+  arrow::Result<std::shared_ptr<arrow::io::OutputStream>> maybe_file_output_stream =
+      io::FileOutputStream::Open(filename, &((*feather_writer)->file_output_stream_));
+  RETURN_NOT_OK(maybe_file_output_stream);
+  (*feather_writer)->file_output_stream_ = maybe_file_output_stream.ValueOrDie();

Review comment:
       will do.




-- 
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] kou commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/CMakeLists.txt
##########
@@ -29,22 +30,51 @@ if(EXISTS "${CPP_CMAKE_MODULES}")
   set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CPP_CMAKE_MODULES})
 endif()
 
-## Arrow is Required
+set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CMAKE_SOURCE_DIR}/cmake_modules)
+
+# Arrow is Required
 find_package(Arrow REQUIRED)
 
-## MATLAB is required to be installed to build MEX interfaces
-set(MATLAB_ADDITIONAL_VERSIONS "R2018a=9.4")
-find_package(Matlab REQUIRED MX_LIBRARY)
-
-# Build featherread mex file based on the arrow shared library
-matlab_add_mex(NAME featherreadmex
-               SRC src/featherreadmex.cc src/feather_reader.cc src/util/handle_status.cc
-                   src/util/unicode_conversion.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherreadmex PRIVATE ${ARROW_INCLUDE_DIR})
-
-# Build featherwrite mex file based on the arrow shared library
-matlab_add_mex(NAME featherwritemex
-               SRC src/featherwritemex.cc src/feather_writer.cc src/util/handle_status.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherwritemex PRIVATE ${ARROW_INCLUDE_DIR})
+# MATLAB is Required
+find_package(Matlab REQUIRED)
+
+# Construct the absolute path to featherread's source files
+set(featherread_sources featherreadmex.cc feather_reader.cc util/handle_status.cc
+                        util/unicode_conversion.cc)
+list(TRANSFORM featherread_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherreadmex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherreadmex
+               SRC
+               ${featherread_sources}
+               LINK_TO
+               arrow_shared)
+
+# Construct the absolute path to featherwrite's source files
+set(featherwrite_sources featherwritemex.cc feather_writer.cc util/handle_status.cc
+                         util/unicode_conversion.cc)
+list(TRANSFORM featherwrite_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherwritemex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherwritemex
+               SRC
+               ${featherwrite_sources}
+               LINK_TO
+               arrow_shared)
+
+# Ensure the MEX binaries are placed in the src directory on all platforms

Review comment:
       Thanks for describing this.
   Could you try one of them (or both)? If we can find a better approach, we can choose it.
   It can be done in this pull request or a follow up task.
   




-- 
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] kou commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Thanks. Don't worry. It's not a your problem.


-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_writer.h
##########
@@ -17,40 +17,36 @@
 
 #pragma once
 
-#include <memory>
-#include <string>
-
 #include <arrow/ipc/feather.h>
 #include <arrow/status.h>
 #include <arrow/type.h>
-
 #include <matrix.h>
 
+#include <memory>
+#include <string>

Review comment:
       Nope. See my comment below. I'll move them back to the top.




-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/CMakeLists.txt
##########
@@ -15,7 +15,8 @@
 # specific language governing permissions and limitations
 # under the License.
 
-cmake_minimum_required(VERSION 3.2)
+cmake_minimum_required(VERSION 3.20)

Review comment:
       We wanted to use 3.20 because earlier versions of the FindMatlab.cmake module had a few bugs we ran into when trying to build our mex functions. I'm open to other approaches, but the hope was to build on the existing work of the cmake community to avoid reinventing the wheel. We also want to share improvements upstream where appropriate. 




-- 
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] sgilmore10 commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Hi @kou,
   
   Regarding CI, we're currently working on adding GoogleTest support in this branch [here](https://github.com/mathworks/arrow/tree/ARROW-13100) if you're interested. Once this pull request is accepted, we can submit a pull request for this branch. That branch requires the changes here to build, which is why we haven't opened pull request yet.
   
   Regarding MATLAB CI, we're actively working on this and will open a pull request as soon as possible.
   
   Best,
   Sarah


-- 
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] wesm commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,34 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
+  RETURN_NOT_OK(maybe_readable_file);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  std::shared_ptr<io::RandomAccessFile> random_access_file{
+      maybe_readable_file.ValueOrDie()};

Review comment:
       Why not use `ARROW_ASSIGN_OR_RAISE` here?

##########
File path: matlab/CMakeLists.txt
##########
@@ -15,7 +15,8 @@
 # specific language governing permissions and limitations
 # under the License.
 
-cmake_minimum_required(VERSION 3.2)
+cmake_minimum_required(VERSION 3.20)

Review comment:
       This seems a bit high, is it necessary?

##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +331,46 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);
+    std::shared_ptr<arrow::Field> field =
+        std::make_shared<arrow::Field>(name_str, datatype);
+
+    arrow::Result<std::shared_ptr<ResizableBuffer>> maybe_buffer =
+        arrow::AllocateResizableBuffer(internal::BitPackedLength(num_rows_));
+    RETURN_NOT_OK(maybe_buffer);
+    std::shared_ptr<ResizableBuffer> validity_bitmap = maybe_buffer.ValueOrDie();

Review comment:
       ARROW_ASSIGN_OR_RAISE?

##########
File path: matlab/src/featherwritemex.cc
##########
@@ -15,10 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include <mex.h>
 
+#include <string>
+

Review comment:
       ?

##########
File path: matlab/src/feather_reader.cc
##########
@@ -242,16 +245,34 @@ mxArray* FeatherReader::ReadVariables() const {
   mxArray* variables =
       mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
 
-  // Read all the table variables in the Feather file into memory.
+  // Read the entire table in the Feather file into memory.
+  std::shared_ptr<arrow::Table> table = nullptr;
+  arrow::Status status = reader_->Read(&table);
+  if (!status.ok()) {
+    mexErrMsgIdAndTxt("MATLAB:arrow:FeatherReader::FailedToReadTable",
+                      "Failed to read arrow::Table from Feather file.");

Review comment:
       do you want to append the Status message? You might consider having a utility to adorn a status with additional Matlab-specific error message to make this pattern simpler

##########
File path: matlab/src/feather_writer.h
##########
@@ -17,40 +17,36 @@
 
 #pragma once
 
-#include <memory>
-#include <string>
-
 #include <arrow/ipc/feather.h>
 #include <arrow/status.h>
 #include <arrow/type.h>
-
 #include <matrix.h>
 
+#include <memory>
+#include <string>

Review comment:
       Is there a reason to move these here?

##########
File path: matlab/src/feather_writer.cc
##########
@@ -316,22 +331,46 @@ Status FeatherWriter::WriteVariables(const mxArray* variables) {
     std::string name_str = internal::MxArrayToString(name);
     std::string type_str = internal::MxArrayToString(type);
 
+    std::shared_ptr<arrow::DataType> datatype =
+        internal::ConvertMatlabTypeStringToArrowDataType(type_str);
+    std::shared_ptr<arrow::Field> field =
+        std::make_shared<arrow::Field>(name_str, datatype);
+
+    arrow::Result<std::shared_ptr<ResizableBuffer>> maybe_buffer =
+        arrow::AllocateResizableBuffer(internal::BitPackedLength(num_rows_));
+    RETURN_NOT_OK(maybe_buffer);
+    std::shared_ptr<ResizableBuffer> validity_bitmap = maybe_buffer.ValueOrDie();
+
     // Populate bit-packed arrow::Buffer using validity data in the mxArray*.
     internal::BitPackBuffer(valid, validity_bitmap);
 
     // Wrap mxArray data in an arrow::Array of the equivalent type.
-    std::unique_ptr<Array> array =
+    std::shared_ptr<Array> array =
         internal::WriteVariableData(data, type_str, validity_bitmap);
 
     // Verify that the arrow::Array has the right number of elements.
-    internal::ValidateNumRows(array->length(), this->num_rows_);
+    internal::ValidateNumRows(array->length(), num_rows_);
+
+    // Append the field to the schema builder
+    RETURN_NOT_OK(schema_builder.AddField(field));
 
-    // Write another column to the Feather file.
-    ARROW_RETURN_NOT_OK(this->table_writer_->Append(name_str, *array));
+    // Store the table column
+    table_columns.push_back(array);
   }
+  // Create the table schema
+  arrow::Result<std::shared_ptr<arrow::Schema>> table_schema_result =
+      schema_builder.Finish();
+  RETURN_NOT_OK(table_schema_result);
+
+  std::shared_ptr<arrow::Schema> table_schema = table_schema_result.ValueOrDie();

Review comment:
       ARROW_ASSIGN_OR_RAISE?

##########
File path: matlab/src/feather_writer.cc
##########
@@ -26,18 +24,53 @@
 #include <arrow/status.h>
 #include <arrow/table.h>
 #include <arrow/type.h>
-#include <arrow/util/bit-util.h>
-
+#include <arrow/util/bit_util.h>
+#include <arrow/util/bitmap_generate.h>
+#include <arrow/util/key_value_metadata.h>
 #include <mex.h>
 
-#include "feather_writer.h"
+#include <cmath>
+#include <functional> /* for std::multiplies */
+#include <numeric>    /* for std::accumulate */

Review comment:
       We generally put stdlib includes before third party includes

##########
File path: matlab/src/featherreadmex.cc
##########
@@ -15,10 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include <mex.h>
 
+#include <string>
+

Review comment:
       reason for this change?

##########
File path: matlab/src/feather_writer.cc
##########
@@ -248,60 +279,44 @@ Status FeatherWriter::Open(const std::string& filename,
   *feather_writer = std::shared_ptr<FeatherWriter>(new FeatherWriter());
 
   // Open a FileOutputStream corresponding to the provided filename.
-  std::shared_ptr<io::OutputStream> writable_file(nullptr);
-  ARROW_RETURN_NOT_OK(io::FileOutputStream::Open(filename, &writable_file));
-
-  // TableWriter::Open expects a shared_ptr to an OutputStream.
-  // Open the Feather file for writing with a TableWriter.
-  return ipc::feather::TableWriter::Open(writable_file,
-                                         &(*feather_writer)->table_writer_);
-}
-
-// Write table metadata to the Feather file from a mxArray*.
-void FeatherWriter::WriteMetadata(const mxArray* metadata) {
-  // Verify that all required fieldnames are provided.
-  internal::ValidateMxStructField(metadata, "Description", mxCHAR_CLASS, true);
-  internal::ValidateMxStructField(metadata, "NumRows", mxDOUBLE_CLASS, false);
-  internal::ValidateMxStructField(metadata, "NumVariables", mxDOUBLE_CLASS, false);
-
-  // Convert Description to a std::string and set on FeatherWriter and TableWriter.
-  std::string description =
-      internal::MxArrayToString(mxGetField(metadata, 0, "Description"));
-  this->description_ = description;
-  this->table_writer_->SetDescription(description);
-
-  // Get the NumRows field in the struct array and set on TableWriter.
-  this->num_rows_ = static_cast<int64_t>(mxGetScalar(mxGetField(metadata, 0, "NumRows")));
-  this->table_writer_->SetNumRows(this->num_rows_);
+  arrow::Result<std::shared_ptr<arrow::io::OutputStream>> maybe_file_output_stream =
+      io::FileOutputStream::Open(filename, &((*feather_writer)->file_output_stream_));
+  RETURN_NOT_OK(maybe_file_output_stream);
+  (*feather_writer)->file_output_stream_ = maybe_file_output_stream.ValueOrDie();

Review comment:
       ARROW_ASSIGN_OR_RAISE?

##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,34 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
+  RETURN_NOT_OK(maybe_readable_file);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  std::shared_ptr<io::RandomAccessFile> random_access_file{
+      maybe_readable_file.ValueOrDie()};
 
   // Open the Feather file for reading with a TableReader.
-  RETURN_NOT_OK(ipc::feather::TableReader::Open(random_access_file,
-                                                &(*feather_reader)->table_reader_));
-
-  // Read the table metadata from the Feather file.
-  (*feather_reader)->num_rows_ = (*feather_reader)->table_reader_->num_rows();
-  (*feather_reader)->num_variables_ = (*feather_reader)->table_reader_->num_columns();
-  (*feather_reader)->description_ =
-      (*feather_reader)->table_reader_->HasDescription()
-          ? (*feather_reader)->table_reader_->GetDescription()
-          : "";
-
-  if ((*feather_reader)->num_rows_ > internal::MAX_MATLAB_SIZE ||
-      (*feather_reader)->num_variables_ > internal::MAX_MATLAB_SIZE) {
-    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
-                      "The table size exceeds MATLAB limits: %u x %u",
-                      (*feather_reader)->num_rows_, (*feather_reader)->num_variables_);
+  arrow::Result<std::shared_ptr<ipc::feather::Reader>> maybe_reader =
+      ipc::feather::Reader::Open(random_access_file);
+  RETURN_NOT_OK(maybe_reader);
+
+  // Set the internal reader_ object.
+  (*feather_reader)->reader_ = maybe_reader.ValueOrDie();

Review comment:
       `ARROW_ASSIGN_OR_RAISE`?

##########
File path: matlab/src/feather_reader.cc
##########
@@ -242,16 +245,34 @@ mxArray* FeatherReader::ReadVariables() const {
   mxArray* variables =
       mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
 
-  // Read all the table variables in the Feather file into memory.
+  // Read the entire table in the Feather file into memory.
+  std::shared_ptr<arrow::Table> table = nullptr;

Review comment:
       redundant




-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/featherreadmex.cc
##########
@@ -15,10 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include <mex.h>
 
+#include <string>
+

Review comment:
       I was trying to reorganize the include statements in a way that made sense to me, but I'd rather go for consistency with the rest of the code base. I'll move the stdlib includes up top.




-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/featherwritemex.cc
##########
@@ -15,10 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include <mex.h>
 
+#include <string>
+

Review comment:
       I'll move the include statement to the top.




-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,34 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
+  RETURN_NOT_OK(maybe_readable_file);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  std::shared_ptr<io::RandomAccessFile> random_access_file{
+      maybe_readable_file.ValueOrDie()};

Review comment:
       Will do.




-- 
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] kou commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Thanks!


-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_reader.cc
##########
@@ -177,32 +182,32 @@ Status FeatherReader::Open(const std::string& filename,
   *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
 
   // Open file with given filename as a ReadableFile.
-  std::shared_ptr<io::ReadableFile> readable_file(nullptr);
-
-  RETURN_NOT_OK(io::ReadableFile::Open(filename, &readable_file));
+  arrow::Result<std::shared_ptr<io::ReadableFile>> maybe_readable_file =
+      io::ReadableFile::Open(filename);
 
   // TableReader expects a RandomAccessFile.
-  std::shared_ptr<io::RandomAccessFile> random_access_file(readable_file);
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<io::RandomAccessFile> random_access_file,
+                        maybe_readable_file);
 
   // Open the Feather file for reading with a TableReader.
-  RETURN_NOT_OK(ipc::feather::TableReader::Open(random_access_file,
-                                                &(*feather_reader)->table_reader_));
-
-  // Read the table metadata from the Feather file.
-  (*feather_reader)->num_rows_ = (*feather_reader)->table_reader_->num_rows();
-  (*feather_reader)->num_variables_ = (*feather_reader)->table_reader_->num_columns();
-  (*feather_reader)->description_ =
-      (*feather_reader)->table_reader_->HasDescription()
-          ? (*feather_reader)->table_reader_->GetDescription()
-          : "";
-
-  if ((*feather_reader)->num_rows_ > internal::MAX_MATLAB_SIZE ||
-      (*feather_reader)->num_variables_ > internal::MAX_MATLAB_SIZE) {
-    mexErrMsgIdAndTxt("MATLAB:arrow:SizeTooLarge",
-                      "The table size exceeds MATLAB limits: %u x %u",
-                      (*feather_reader)->num_rows_, (*feather_reader)->num_variables_);
+  arrow::Result<std::shared_ptr<ipc::feather::Reader>> maybe_reader =
+      ipc::feather::Reader::Open(random_access_file);
+  ARROW_ASSIGN_OR_RAISE(auto reader, maybe_reader);

Review comment:
       Just made this change (with auto) in separate commit.




-- 
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] sgilmore10 commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Sorry about that. Just rebased on master.


-- 
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] sgilmore10 commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Thanks for the feedback @wesm! I'll make some changes and push to the pr once I'm done. 
   
   Best,
   Sarah


-- 
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] sgilmore10 commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Thanks for looking at this @lidavidm! I rebased against the master branch.
   
   Ideally, we would like to get reviewers outside of MathWorks. I'm not sure what the precedent is for new language bindings in terms of finding the right reviewers.  We understand it takes a fair bit of time and effort to review pull requests, so let us know if there's anything we can do to help.
   
   Best,
   Sarah
   


-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_reader.cc
##########
@@ -52,11 +57,12 @@ mxArray* ReadNumericVariableData(const std::shared_ptr<Array>& column) {
   mxArray* variable_data =
       mxCreateNumericMatrix(column->length(), 1, matlab_class_id, mxREAL);
 
-  std::shared_ptr<ArrowArrayType> integer_array =
+  std::shared_ptr<ArrowArrayType> arrow_numeric_array =
       std::static_pointer_cast<ArrowArrayType>(column);
 
   // Get a raw pointer to the Arrow array data.
-  const MatlabType* source = integer_array->raw_values();
+  const MatlabType* source =
+      reinterpret_cast<const MatlabType*>(arrow_numeric_array->values()->data());

Review comment:
       From the documentation, it looks like the values() method doesn't account for slice offsets. I think we modified this line when we were trying to get the code to compile again, but didn't check to see if this change was really necessary. I just tried using raw_values() instead and it works. I'll undo this 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.

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 pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Sorry. This is conflicted caused by #10571. Could you rebase on the master?


-- 
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] sgilmore10 commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   No worries!  I just messaged the mailing list to see if anyone would be interested in reviewing this. 
   
   Best,
   Sarah
   


-- 
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 #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Hey, sorry for the delay (again) - I can try to review this once I get a chance, but you may also want to poke the mailing list (dev@arrow.apache.org) to see if someone else is interested.


-- 
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 #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


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


-- 
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] kou closed pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   


-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/src/feather_reader.cc
##########
@@ -242,16 +245,34 @@ mxArray* FeatherReader::ReadVariables() const {
   mxArray* variables =
       mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
 
-  // Read all the table variables in the Feather file into memory.
+  // Read the entire table in the Feather file into memory.
+  std::shared_ptr<arrow::Table> table = nullptr;
+  arrow::Status status = reader_->Read(&table);
+  if (!status.ok()) {
+    mexErrMsgIdAndTxt("MATLAB:arrow:FeatherReader::FailedToReadTable",
+                      "Failed to read arrow::Table from Feather file.");

Review comment:
       That's a good point. I'll update the error to include the status message. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [arrow] sgilmore10 commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   I'm not too familiar with Travis CI, but I believe the build failure failure on ARM may be unrelated to my changes and/or a pre-existing build issue. It looks like other failed jobs have experienced a similar issue on ARM as well. If there's anything I need to do, please let me know.


-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/CMakeLists.txt
##########
@@ -29,22 +30,51 @@ if(EXISTS "${CPP_CMAKE_MODULES}")
   set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CPP_CMAKE_MODULES})
 endif()
 
-## Arrow is Required
+set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CMAKE_SOURCE_DIR}/cmake_modules)
+
+# Arrow is Required
 find_package(Arrow REQUIRED)
 
-## MATLAB is required to be installed to build MEX interfaces
-set(MATLAB_ADDITIONAL_VERSIONS "R2018a=9.4")
-find_package(Matlab REQUIRED MX_LIBRARY)
-
-# Build featherread mex file based on the arrow shared library
-matlab_add_mex(NAME featherreadmex
-               SRC src/featherreadmex.cc src/feather_reader.cc src/util/handle_status.cc
-                   src/util/unicode_conversion.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherreadmex PRIVATE ${ARROW_INCLUDE_DIR})
-
-# Build featherwrite mex file based on the arrow shared library
-matlab_add_mex(NAME featherwritemex
-               SRC src/featherwritemex.cc src/feather_writer.cc src/util/handle_status.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherwritemex PRIVATE ${ARROW_INCLUDE_DIR})
+# MATLAB is Required
+find_package(Matlab REQUIRED)
+
+# Construct the absolute path to featherread's source files
+set(featherread_sources featherreadmex.cc feather_reader.cc util/handle_status.cc
+                        util/unicode_conversion.cc)
+list(TRANSFORM featherread_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherreadmex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherreadmex
+               SRC
+               ${featherread_sources}
+               LINK_TO
+               arrow_shared)
+
+# Construct the absolute path to featherwrite's source files
+set(featherwrite_sources featherwritemex.cc feather_writer.cc util/handle_status.cc
+                         util/unicode_conversion.cc)
+list(TRANSFORM featherwrite_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherwritemex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherwritemex
+               SRC
+               ${featherwrite_sources}
+               LINK_TO
+               arrow_shared)
+
+# Ensure the MEX binaries are placed in the src directory on all platforms

Review comment:
       We could add the build folder to the MATLAB search path. However, this will make harder to run our unit tests automatically because we would require the user to explicitly tell us where their build files are located every time. 
   
   Additionally, as the MATLAB interface grows, we will have many MEX files. In order to keep things organized. We see two approaches:
   
   1) We can encode the relationship of MEX files to MATLAB classes via the MEX file name itself. For example, arrow_array_new.mexw64. 
   
   or 
   
   2) The other option is to encode the relationship between the two via the MATLAB packaging mechanism. For example, the folder structure +matlab/+array/+mex exposes a package called matlab.array.mex to MATLAB. If we choose this option, we can reuse common names, such as "new", and keep the names short. 
   
   Option 2 seems more scalable, but if you're experience tells us this will lead to issues, we can revisit adding the build folder to the path. 
   
   Best,
   Sarah
   




-- 
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] sgilmore10 commented on pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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


   Just pushed changes based on the code review feedback.


-- 
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] sgilmore10 commented on a change in pull request #10305: ARROW-12730: [MATLAB] Update featherreadmex and featherwritemex to build against latest Arrow C++ APIs

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



##########
File path: matlab/CMakeLists.txt
##########
@@ -29,22 +30,51 @@ if(EXISTS "${CPP_CMAKE_MODULES}")
   set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CPP_CMAKE_MODULES})
 endif()
 
-## Arrow is Required
+set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CMAKE_SOURCE_DIR}/cmake_modules)
+
+# Arrow is Required
 find_package(Arrow REQUIRED)
 
-## MATLAB is required to be installed to build MEX interfaces
-set(MATLAB_ADDITIONAL_VERSIONS "R2018a=9.4")
-find_package(Matlab REQUIRED MX_LIBRARY)
-
-# Build featherread mex file based on the arrow shared library
-matlab_add_mex(NAME featherreadmex
-               SRC src/featherreadmex.cc src/feather_reader.cc src/util/handle_status.cc
-                   src/util/unicode_conversion.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherreadmex PRIVATE ${ARROW_INCLUDE_DIR})
-
-# Build featherwrite mex file based on the arrow shared library
-matlab_add_mex(NAME featherwritemex
-               SRC src/featherwritemex.cc src/feather_writer.cc src/util/handle_status.cc
-               LINK_TO ${ARROW_SHARED_LIB})
-target_include_directories(featherwritemex PRIVATE ${ARROW_INCLUDE_DIR})
+# MATLAB is Required
+find_package(Matlab REQUIRED)
+
+# Construct the absolute path to featherread's source files
+set(featherread_sources featherreadmex.cc feather_reader.cc util/handle_status.cc
+                        util/unicode_conversion.cc)
+list(TRANSFORM featherread_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherreadmex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherreadmex
+               SRC
+               ${featherread_sources}
+               LINK_TO
+               arrow_shared)
+
+# Construct the absolute path to featherwrite's source files
+set(featherwrite_sources featherwritemex.cc feather_writer.cc util/handle_status.cc
+                         util/unicode_conversion.cc)
+list(TRANSFORM featherwrite_sources PREPEND ${CMAKE_SOURCE_DIR}/src/)
+
+# Build featherwritemex MEX binary
+matlab_add_mex(R2018a
+               NAME
+               featherwritemex
+               SRC
+               ${featherwrite_sources}
+               LINK_TO
+               arrow_shared)
+
+# Ensure the MEX binaries are placed in the src directory on all platforms

Review comment:
       In order to execute the MEX function, it has to be discoverable on the MATLAB search path. We also have MATLAB code (featherread.m and featherwrite.m) that we also need to add to the MATLAB search path. Since we need to add the source directory to the path anyway, I thought it makes sense to put the MEX files there as well. 




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