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

[arrow] branch master updated: ARROW-2750: [MATLAB] Initial MATLAB interface, support for reading numeric types from Feather files

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f397ce9  ARROW-2750: [MATLAB] Initial MATLAB interface, support for reading numeric types from Feather files
f397ce9 is described below

commit f397ce98ece24fbec5eea0734e274621c5b81b38
Author: Kevin Gurney <ke...@gmail.com>
AuthorDate: Sat Sep 8 13:38:24 2018 -0400

    ARROW-2750: [MATLAB] Initial MATLAB interface, support for reading numeric types from Feather files
    
    Reopening #2172 from @kevingurney
    
    Add MATLAB support for reading numeric types (i.e. [u]int[x][y], float, double) from Feather files. This is the first in a series of future feature submissions for Feather read/write support and other Arrow IPC integration with MATLAB.
    
    The associated pull request creates a top-level "matlab" directory in the Apache Arrow project. It also introduces a MATLAB function "featherread", which takes a Feather filename as input and returns a MATLAB table. featherread maps Feather datatypes to corresponding MATLAB datatypes.
    
    This initial pull request does not support null values.
    
    featherread.m calls the Arrow C+ APIs using MEX, MATLAB's facility for calling C/C+ code.
    
    See the README.md in the "matlab" directory for instructions on how to build the MEX interface. Currently, building on Windows using CMake is not fully functional, but the MEX interface can be compiled manually using the MATLAB "mex" command.
    
    A MATLAB install is required to be present on your machine to build the MEX interface.
    
    Author: Kevin Gurney <ke...@gmail.com>
    
    Closes #2208 from wesm/mathworks-master and squashes the following commits:
    
    ce1892dc7 <Kevin Gurney> Address code review feedback on pull request #2208. Add instructions for running tests to README.md.
    3059c2f17 <Kevin Gurney> Add Apache License Version 2.0 to notAFeatherFile.feather.
    900a3362f <Kevin Gurney> ARROW-2750: Add MATLAB support for reading numeric types from Feather files
---
 matlab/.gitignore                                  |  23 +++
 matlab/CMakeLists.txt                              |  42 ++++
 matlab/README.md                                   |  79 ++++++++
 matlab/src/feather_reader.cc                       | 212 +++++++++++++++++++++
 matlab/src/feather_reader.h                        |  71 +++++++
 matlab/src/featherread.m                           |  93 +++++++++
 matlab/src/featherreadmex.cc                       |  36 ++++
 matlab/src/matlab_traits.h                         | 103 ++++++++++
 matlab/src/util/handle_status.cc                   | 110 +++++++++++
 matlab/src/util/handle_status.h                    |  32 ++++
 matlab/test/corrupted_feather_file.feather         |   5 +
 matlab/test/not_a_feather_file.feather             |  18 ++
 ...ic_datatypes_6th_variable_name_is_empty.feather | Bin 0 -> 976 bytes
 .../test/numeric_datatypes_with_nan_column.feather | Bin 0 -> 1040 bytes
 matlab/test/numeric_datatypes_with_nan_row.feather | Bin 0 -> 1272 bytes
 .../test/numeric_datatypes_with_no_nulls.feather   | Bin 0 -> 984 bytes
 matlab/test/tfeatherread.m                         | 141 ++++++++++++++
 17 files changed, 965 insertions(+)

diff --git a/matlab/.gitignore b/matlab/.gitignore
new file mode 100644
index 0000000..e89b1b9
--- /dev/null
+++ b/matlab/.gitignore
@@ -0,0 +1,23 @@
+# 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.
+
+# CMake files
+CMakeFiles/*
+CMakeCache.txt
+
+# MEX files
+*.mex*
diff --git a/matlab/CMakeLists.txt b/matlab/CMakeLists.txt
new file mode 100755
index 0000000..8970866
--- /dev/null
+++ b/matlab/CMakeLists.txt
@@ -0,0 +1,42 @@
+# 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.
+
+cmake_minimum_required(VERSION 3.2)
+set(CMAKE_CXX_STANDARD 11)
+
+project(mlarrow)
+
+# Grab CMAKE Modules from the CPP interface
+set(CPP_CMAKE_MODULES "${CMAKE_SOURCE_DIR}/../cpp/cmake_modules")
+if(EXISTS "${CPP_CMAKE_MODULES}")
+  set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} ${CPP_CMAKE_MODULES})
+endif()
+
+## 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
+               LINK_TO ${ARROW_SHARED_LIB})
+target_include_directories(featherreadmex PRIVATE ${ARROW_INCLUDE_DIR})
diff --git a/matlab/README.md b/matlab/README.md
new file mode 100644
index 0000000..663bd20
--- /dev/null
+++ b/matlab/README.md
@@ -0,0 +1,79 @@
+<!---
+  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.
+-->
+
+## MATLAB Library for Apache Arrow
+
+## Status
+
+This is a very early stage MATLAB interface to the Apache Arrow C++ libraries.
+
+The current code only supports reading numeric types from Feather files.
+
+## Building from source
+
+### Get Arrow and build Arrow CPP
+
+See: [Arrow CPP README](../cpp/README.md)
+
+### Build MATLAB interface to Apache Arrow using MATLAB R2018a or later:
+
+    cd arrow/matlab
+    mkdir build
+    cd build
+    cmake ..
+    make
+
+### Non-standard MATLAB and Arrow installations
+
+To specify a non-standard MATLAB install location, use the Matlab_ROOT_DIR CMake flag:
+
+    cmake .. -DMatlab_ROOT_DIR=/<PATH_TO_MATLAB_INSTALL>
+
+To specify a non-standard Arrow install location, use the ARROW_HOME CMake flag:
+
+    cmake .. -DARROW_HOME=/<PATH_TO_ARROW_INSTALL>
+
+## Try it out
+
+### Add the src and build directories to your MATLAB path
+
+``` matlab
+>> cd(fullfile('arrow', 'matlab'));
+>> addpath src;
+>> addpath build;
+```
+
+### Read a Feather file into MATLAB as a table
+
+``` matlab
+>> filename = fullfile('arrow', 'matlab', 'test', 'numericDatatypesWithNoNulls.feather');
+>> t = featherread(filename);
+```
+
+This should return a MATLAB table datatype containing the Feather file contents.
+
+## Running the tests
+
+``` matlab
+>> cd(fullfile('arrow', 'matlab'));
+>> addpath src;
+>> addpath build;
+>> cd test;
+>> runtests .;
+```
diff --git a/matlab/src/feather_reader.cc b/matlab/src/feather_reader.cc
new file mode 100644
index 0000000..22bbc66
--- /dev/null
+++ b/matlab/src/feather_reader.cc
@@ -0,0 +1,212 @@
+// 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/io/file.h>
+#include <arrow/ipc/feather.h>
+#include <arrow/status.h>
+#include <arrow/table.h>
+#include <arrow/type.h>
+
+#include <mex.h>
+
+#include "feather_reader.h"
+#include "matlab_traits.h"
+#include "util/handle_status.h"
+
+namespace mlarrow {
+
+namespace internal {
+
+// Read the name of variable i from the Feather file as a mxArray*.
+mxArray* ReadVariableName(const std::shared_ptr<arrow::Column>& column) {
+  return mxCreateString(column->name().c_str());
+}
+
+template <typename ArrowDataType>
+mxArray* ReadNumericVariableData(const std::shared_ptr<arrow::Column>& column) {
+  typedef typename MatlabTraits<ArrowDataType>::MatlabType MatlabType;
+  typedef typename arrow::TypeTraits<ArrowDataType>::ArrayType ArrowArrayType;
+
+  std::shared_ptr<arrow::ChunkedArray> chunked_array = column->data();
+  const int num_chunks = chunked_array->num_chunks();
+
+  const mxClassID matlab_class_id = MatlabTraits<ArrowDataType>::matlab_class_id;
+  // Allocate a numeric mxArray* with the correct mxClassID based on the type of the
+  // arrow::Column.
+  mxArray* variable_data =
+      mxCreateNumericMatrix(column->length(), 1, matlab_class_id, mxREAL);
+
+  int64_t mx_array_offset = 0;
+  // Iterate over each arrow::Array in the arrow::ChunkedArray.
+  for (int i = 0; i < num_chunks; ++i) {
+    std::shared_ptr<arrow::Array> array = chunked_array->chunk(i);
+    const int64_t chunk_length = array->length();
+    std::shared_ptr<ArrowArrayType> arr = std::static_pointer_cast<ArrowArrayType>(array);
+    const auto data = arr->raw_values();
+    MatlabType* dt = MatlabTraits<ArrowDataType>::GetData(variable_data);
+    std::copy(data, data + chunk_length, dt + mx_array_offset);
+    mx_array_offset += chunk_length;
+  }
+
+  return variable_data;
+}
+
+// Read the data of variable i from the Feather file as a mxArray*.
+mxArray* ReadVariableData(const std::shared_ptr<arrow::Column>& column) {
+  std::shared_ptr<arrow::DataType> type = column->type();
+
+  switch (type->id()) {
+    case arrow::Type::FLOAT:
+      return ReadNumericVariableData<arrow::FloatType>(column);
+    case arrow::Type::DOUBLE:
+      return ReadNumericVariableData<arrow::DoubleType>(column);
+    case arrow::Type::UINT8:
+      return ReadNumericVariableData<arrow::UInt8Type>(column);
+    case arrow::Type::UINT16:
+      return ReadNumericVariableData<arrow::UInt16Type>(column);
+    case arrow::Type::UINT32:
+      return ReadNumericVariableData<arrow::UInt32Type>(column);
+    case arrow::Type::UINT64:
+      return ReadNumericVariableData<arrow::UInt64Type>(column);
+    case arrow::Type::INT8:
+      return ReadNumericVariableData<arrow::Int8Type>(column);
+    case arrow::Type::INT16:
+      return ReadNumericVariableData<arrow::Int16Type>(column);
+    case arrow::Type::INT32:
+      return ReadNumericVariableData<arrow::Int32Type>(column);
+    case arrow::Type::INT64:
+      return ReadNumericVariableData<arrow::Int64Type>(column);
+
+    default: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:UnsupportedArrowType",
+                        "Unsupported arrow::Type '%s' for variable '%s'",
+                        type->name().c_str(), column->name().c_str());
+      break;
+    }
+  }
+
+  return nullptr;
+}
+
+// Read the nulls of variable i from the Feather file as a mxArray*.
+mxArray* ReadVariableNulls(const std::shared_ptr<arrow::Column>& column) {
+  // TODO: Implement proper null value support. For the time being,
+  // we will simply return a zero initialized logical array to MATLAB.
+  return mxCreateLogicalMatrix(column->length(), 1);
+}
+
+// Read the type of variable i from the Feather file as a mxArray*.
+mxArray* ReadVariableType(const std::shared_ptr<arrow::Column>& column) {
+  return mxCreateString(column->type()->name().c_str());
+}
+
+// MATLAB arrays cannot be larger than 2^48.
+static constexpr uint64_t MAX_MATLAB_SIZE = static_cast<uint64_t>(0x01) << 48;
+
+}  // namespace internal
+
+arrow::Status FeatherReader::Open(const std::string& filename,
+                                  std::shared_ptr<FeatherReader>* feather_reader) {
+  *feather_reader = std::shared_ptr<FeatherReader>(new FeatherReader());
+  // Open file with given filename as a ReadableFile.
+  std::shared_ptr<arrow::io::ReadableFile> readable_file(nullptr);
+  auto status = arrow::io::ReadableFile::Open(filename, &readable_file);
+  if (!status.ok()) {
+    return status;
+  }
+  // TableReader expects a RandomAccessFile.
+  std::shared_ptr<arrow::io::RandomAccessFile> random_access_file(readable_file);
+  // Open the Feather file for reading with a TableReader.
+  status = arrow::ipc::feather::TableReader::Open(random_access_file,
+                                                  &(*feather_reader)->table_reader_);
+  if (!status.ok()) {
+    return status;
+  }
+
+  // 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()
+          : "";
+  (*feather_reader)->version_ = (*feather_reader)->table_reader_->version();
+
+  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_);
+  }
+
+  return status;
+}
+
+// Read the table metadata from the Feather file as a mxArray*.
+mxArray* FeatherReader::ReadMetadata() const {
+  const int num_metadata_fields = 4;
+  const char* fieldnames[] = {"NumRows", "NumVariables", "Description", "Version"};
+
+  // Create a mxArray struct array containing the table metadata to be passed back to
+  // MATLAB.
+  mxArray* metadata = mxCreateStructMatrix(1, 1, num_metadata_fields, fieldnames);
+
+  // Returning double values to MATLAB since that is the default type.
+
+  // Set the number of rows.
+  mxSetField(metadata, 0, "NumRows",
+             mxCreateDoubleScalar(static_cast<double>(num_rows_)));
+
+  // Set the number of variables.
+  mxSetField(metadata, 0, "NumVariables",
+             mxCreateDoubleScalar(static_cast<double>(num_variables_)));
+
+  // Set the description.
+  mxSetField(metadata, 0, "Description", mxCreateString(description_.c_str()));
+
+  // Set the version.
+  mxSetField(metadata, 0, "Version", mxCreateDoubleScalar(static_cast<double>(version_)));
+
+  return metadata;
+}
+
+// Read the table variables from the Feather file as a mxArray*.
+mxArray* FeatherReader::ReadVariables() const {
+  const int num_variable_fields = 4;
+  const char* fieldnames[] = {"Name", "Data", "Nulls", "Type"};
+
+  // Create an mxArray struct array containing the table variables to be passed back to
+  // MATLAB.
+  mxArray* variables =
+      mxCreateStructMatrix(1, num_variables_, num_variable_fields, fieldnames);
+
+  // Read all the table variables in the Feather file into memory.
+  for (int64_t i = 0; i < num_variables_; ++i) {
+    std::shared_ptr<arrow::Column> column(nullptr);
+    util::HandleStatus(table_reader_->GetColumn(i, &column));
+
+    // set the struct fields data
+    mxSetField(variables, i, "Name", internal::ReadVariableName(column));
+    mxSetField(variables, i, "Data", internal::ReadVariableData(column));
+    mxSetField(variables, i, "Nulls", internal::ReadVariableNulls(column));
+    mxSetField(variables, i, "Type", internal::ReadVariableType(column));
+  }
+
+  return variables;
+}
+
+}  // namespace mlarrow
diff --git a/matlab/src/feather_reader.h b/matlab/src/feather_reader.h
new file mode 100644
index 0000000..51405b5
--- /dev/null
+++ b/matlab/src/feather_reader.h
@@ -0,0 +1,71 @@
+// 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.
+
+#ifndef MLARROW_FEATHER_READER_H
+#define MLARROW_FEATHER_READER_H
+
+#include <memory>
+#include <string>
+
+#include <arrow/ipc/feather.h>
+#include <arrow/status.h>
+#include <arrow/type.h>
+
+#include <matrix.h>
+
+namespace mlarrow {
+
+class FeatherReader {
+ public:
+  ~FeatherReader() = default;
+
+  /// \brief Read the table metadata as a mxArray* struct from the given Feather file.
+  ///        The returned struct includes fields describing the number of rows
+  ///        in the table (NumRows), the number of variables (NumVariables), the
+  ///        Feather file version (Version), and the table description (Description).
+  ///        Callers are responsible for freeing the returned mxArray memory
+  ///        when it is no longer needed, or passing it to MATLAB to be managed.
+  /// \return metadata mxArray* scalar struct containing table level metadata
+  mxArray* ReadMetadata() const;
+
+  /// \brief Read the table metadata as a mxArray* struct array from the given
+  ///        Feather file. Each struct includes fields for variable data (Data),
+  ///        null values (Nulls), variable name (Name), and original Feather
+  ///        datatype (Type). Callers are responsible for freeing the returned
+  ///        mxArray memory when it is no longer needed, or passing it to MATLAB
+  ///        to be managed.
+  /// \return variables mxArray* struct array containing table variable data
+  mxArray* ReadVariables() const;
+
+  /// \brief Initialize a FeatherReader object from a given Feather file.
+  /// \param[in] filename path to a Feather file
+  /// \param[out] feather_reader uninitialized FeatherReader object
+  static arrow::Status Open(const std::string& filename,
+                            std::shared_ptr<FeatherReader>* feather_reader);
+
+ private:
+  FeatherReader() = default;
+  std::unique_ptr<arrow::ipc::feather::TableReader> table_reader_;
+  int64_t num_rows_;
+  int64_t num_variables_;
+  std::string description_;
+  int version_;
+};
+
+}  // namespace mlarrow
+
+#endif  // MLARROW_FEATHER_READER_H
diff --git a/matlab/src/featherread.m b/matlab/src/featherread.m
new file mode 100644
index 0000000..ababdcc
--- /dev/null
+++ b/matlab/src/featherread.m
@@ -0,0 +1,93 @@
+function t = featherread(filename)
+%FEATHERREAD Create a table by reading from a Feather file.
+%   Use the FEATHERREAD function to create a table by reading
+%   column-oriented data from a Feather file.
+%
+%   T = FEATHERREAD(FILENAME) creates a table by reading from the Feather
+%   file FILENAME.
+
+% 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.
+
+% Validate input arguments.
+narginchk(1, 1);
+filename = convertStringsToChars(filename);
+if ~ischar(filename)
+    error('MATLAB:arrow:InvalidFilenameDatatype', ...
+        'Filename must be a character vector or string scalar.');
+end
+
+% FOPEN can be used to search for files without an extension on the MATLAB
+% path.
+fid = fopen(filename);
+if fid ~= -1
+    filename = fopen(fid);
+    fclose(fid);
+else
+    error('MATLAB:arrow:UnableToOpenFile', ...
+        'Unable to open file %s.', filename);
+end
+
+% Read table variables and metadata from the given Feather file using
+% libarrow.
+[variables, metadata] = featherreadmex(filename);
+
+% Preallocate a cell array for the case in which
+% the table VariableDescriptions property needs to be modified.
+variableDescriptions = cell(1, numel(variables));
+
+% Iterate over each table variable, handling null entries and invalid
+% variable names appropriately.
+for ii = 1:length(variables)
+    if any(variables(ii).Nulls)
+        switch variables(ii).Type
+            case {'uint8', 'uint16', 'uint32', 'uint64', 'int8', 'int16', 'int32', 'int64'}
+                % MATLAB does not support missing values for integer types, so
+                % cast to double and set missing values to NaN in this case.
+                variables(ii).Data = double(variables(ii).Data);
+        end
+
+        % Set null entries to the appropriate MATLAB missing value using
+        % logical indexing.
+        variables(ii).Data(variables(ii).Nulls) = missing;
+    end
+
+    % Store invalid variable names in the VariableDescriptons
+    % property, and convert any invalid variable names into valid variable
+    % names.
+    setVariableDescriptions = false;
+    if ~isvarname(variables(ii).Name)
+        variableDescriptions{ii} = sprintf('Original variable name: ''%s''', variables(ii).Name);
+        setVariableDescriptions = true;
+    else
+        variableDescriptions{ii} = '';
+    end
+end
+
+% Construct a MATLAB table from the Feather file data.
+t = table(variables.Data, 'VariableNames', matlab.lang.makeValidName({variables.Name}));
+
+% Store original variable names in the VariableDescriptions property
+% if they were modified to be valid MATLAB table variable names.
+if setVariableDescriptions
+    t.Properties.VariableDescriptions = variableDescriptions;
+end
+% Set the Description property of the table based on the Feather file
+% description.
+t.Properties.Description = metadata.Description;
+
+end
diff --git a/matlab/src/featherreadmex.cc b/matlab/src/featherreadmex.cc
new file mode 100644
index 0000000..0499a37
--- /dev/null
+++ b/matlab/src/featherreadmex.cc
@@ -0,0 +1,36 @@
+// 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 <string>
+
+#include <mex.h>
+
+#include "feather_reader.h"
+#include "util/handle_status.h"
+
+// MEX gateway function. This is the entry point for featherreadmex.cpp.
+void mexFunction(int nlhs, mxArray* plhs[], int nrhs, const mxArray* prhs[]) {
+  const std::string filename{mxArrayToUTF8String(prhs[0])};
+
+  // Read the given Feather file into memory.
+  std::shared_ptr<mlarrow::FeatherReader> feather_reader{nullptr};
+  mlarrow::util::HandleStatus(mlarrow::FeatherReader::Open(filename, &feather_reader));
+
+  // Return the Feather file table variables and table metadata to MATLAB.
+  plhs[0] = feather_reader->ReadVariables();
+  plhs[1] = feather_reader->ReadMetadata();
+}
diff --git a/matlab/src/matlab_traits.h b/matlab/src/matlab_traits.h
new file mode 100644
index 0000000..7d5987a
--- /dev/null
+++ b/matlab/src/matlab_traits.h
@@ -0,0 +1,103 @@
+// 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.
+
+#ifndef MLARROW_MATLAB_TRAITS_H
+#define MLARROW_MATLAB_TRAITS_H
+
+#include <arrow/type.h>
+
+#include <matrix.h>
+
+namespace mlarrow {
+
+/// \brief A type traits class mapping Arrow types to MATLAB types.
+template <typename ArrowDataType>
+struct MatlabTraits;
+
+template <>
+struct MatlabTraits<arrow::FloatType> {
+  static constexpr mxClassID matlab_class_id = mxSINGLE_CLASS;
+  typedef mxSingle MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetSingles(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::DoubleType> {
+  static constexpr mxClassID matlab_class_id = mxDOUBLE_CLASS;
+  typedef mxDouble MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetDoubles(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::UInt8Type> {
+  static constexpr mxClassID matlab_class_id = mxUINT8_CLASS;
+  typedef mxUint8 MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetUint8s(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::UInt16Type> {
+  static constexpr mxClassID matlab_class_id = mxUINT16_CLASS;
+  typedef mxUint16 MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetUint16s(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::UInt32Type> {
+  static constexpr mxClassID matlab_class_id = mxUINT32_CLASS;
+  typedef mxUint32 MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetUint32s(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::UInt64Type> {
+  static constexpr mxClassID matlab_class_id = mxUINT64_CLASS;
+  typedef mxUint64 MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetUint64s(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::Int8Type> {
+  static constexpr mxClassID matlab_class_id = mxINT8_CLASS;
+  typedef mxInt8 MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetInt8s(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::Int16Type> {
+  static constexpr mxClassID matlab_class_id = mxINT16_CLASS;
+  typedef mxInt16 MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetInt16s(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::Int32Type> {
+  static constexpr mxClassID matlab_class_id = mxINT32_CLASS;
+  typedef mxInt32 MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetInt32s(pa); }
+};
+
+template <>
+struct MatlabTraits<arrow::Int64Type> {
+  static constexpr mxClassID matlab_class_id = mxINT64_CLASS;
+  typedef mxInt64 MatlabType;
+  static MatlabType* GetData(mxArray* pa) { return mxGetInt64s(pa); }
+};
+
+}  // namespace mlarrow
+
+#endif  // MLARROW_MATLAB_TRAITS_H
diff --git a/matlab/src/util/handle_status.cc b/matlab/src/util/handle_status.cc
new file mode 100644
index 0000000..4b32feb
--- /dev/null
+++ b/matlab/src/util/handle_status.cc
@@ -0,0 +1,110 @@
+// 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/status.h>
+
+#include <mex.h>
+
+namespace mlarrow {
+
+namespace util {
+
+void HandleStatus(const arrow::Status& status) {
+  const char* arrow_error_message = "Arrow error: %s";
+  switch (status.code()) {
+    case arrow::StatusCode::OK: {
+      break;
+    }
+    case arrow::StatusCode::OutOfMemory: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:OutOfMemory", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::KeyError: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:KeyError", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::TypeError: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:TypeError", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::Invalid: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:Invalid", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::IOError: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:IOError", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::CapacityError: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:CapacityError", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::UnknownError: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:UnknownError", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::NotImplemented: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:NotImplemented", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::SerializationError: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:SerializationError", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::PythonError: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:PythonError", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::PlasmaObjectExists: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:PlasmaObjectExists", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::PlasmaObjectNonexistent: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:PlasmaObjectNonexistent",
+                        arrow_error_message, status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::PlasmaStoreFull: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:PlasmaStoreFull", arrow_error_message,
+                        status.ToString().c_str());
+      break;
+    }
+    case arrow::StatusCode::PlasmaObjectAlreadySealed: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:PlasmaObjectAlreadySealed",
+                        arrow_error_message, status.ToString().c_str());
+      break;
+    }
+    default: {
+      mexErrMsgIdAndTxt("MATLAB:arrow:status:UnknownStatus", arrow_error_message,
+                        "Unknown status");
+      break;
+    }
+  }
+}
+}  // namespace util
+}  // namespace mlarrow
diff --git a/matlab/src/util/handle_status.h b/matlab/src/util/handle_status.h
new file mode 100644
index 0000000..18b0773
--- /dev/null
+++ b/matlab/src/util/handle_status.h
@@ -0,0 +1,32 @@
+// 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.
+
+#ifndef MLARROW_UTIL_HANDLE_STATUS_H
+#define MLARROW_UTIL_HANDLE_STATUS_H
+
+#include <arrow/status.h>
+
+namespace mlarrow {
+namespace util {
+// Terminates execution and returns to the MATLAB prompt,
+// displaying an error message if the given status
+// indicates that an error has occurred.
+void HandleStatus(const arrow::Status& status);
+}  // namespace util
+}  // namespace mlarrow
+
+#endif  // MLARROW_UTIL_HANDLE_STATUS_H
diff --git a/matlab/test/corrupted_feather_file.feather b/matlab/test/corrupted_feather_file.feather
new file mode 100755
index 0000000..84c5a5e
--- /dev/null
+++ b/matlab/test/corrupted_feather_file.feather
@@ -0,0 +1,5 @@
+FEW1    ���>  �@UUUUUU�?      @                                	
      
+                                                        
+   �  t    �  �  4  �   �   P      Z���            uint64          ���   `                         ����            uint32          P���   X                         ����            uint16          ����   P                         2���            uint8           ����   H                         z���            int64           (���   8                         ����            int32           p���   0                         
+���            int16           ����   (                         R���   ,         int8             $                                         ����   ,         double           "               
+                                      (                                      	                     X  FEW1
\ No newline at end of file
diff --git a/matlab/test/not_a_feather_file.feather b/matlab/test/not_a_feather_file.feather
new file mode 100755
index 0000000..59f6078
--- /dev/null
+++ b/matlab/test/not_a_feather_file.feather
@@ -0,0 +1,18 @@
+# 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.
+
+This is not a Feather file.
diff --git a/matlab/test/numeric_datatypes_6th_variable_name_is_empty.feather b/matlab/test/numeric_datatypes_6th_variable_name_is_empty.feather
new file mode 100755
index 0000000..6d6dcd6
Binary files /dev/null and b/matlab/test/numeric_datatypes_6th_variable_name_is_empty.feather differ
diff --git a/matlab/test/numeric_datatypes_with_nan_column.feather b/matlab/test/numeric_datatypes_with_nan_column.feather
new file mode 100755
index 0000000..4da6184
Binary files /dev/null and b/matlab/test/numeric_datatypes_with_nan_column.feather differ
diff --git a/matlab/test/numeric_datatypes_with_nan_row.feather b/matlab/test/numeric_datatypes_with_nan_row.feather
new file mode 100755
index 0000000..65a0b50
Binary files /dev/null and b/matlab/test/numeric_datatypes_with_nan_row.feather differ
diff --git a/matlab/test/numeric_datatypes_with_no_nulls.feather b/matlab/test/numeric_datatypes_with_no_nulls.feather
new file mode 100755
index 0000000..37ea6bf
Binary files /dev/null and b/matlab/test/numeric_datatypes_with_no_nulls.feather differ
diff --git a/matlab/test/tfeatherread.m b/matlab/test/tfeatherread.m
new file mode 100755
index 0000000..3e633a6
--- /dev/null
+++ b/matlab/test/tfeatherread.m
@@ -0,0 +1,141 @@
+classdef tfeatherread < matlab.unittest.TestCase
+    % Tests for MATLAB featherread.
+
+    % 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.
+
+    methods(TestClassSetup)
+
+        function addFeatherreadToMATLABPath(testcase)
+            import matlab.unittest.fixtures.PathFixture
+            % Add featherread.m to the MATLAB path.
+            testcase.applyFixture(PathFixture('../'));
+        end
+
+    end
+
+    methods(Test)
+
+        function NumericDatatypesNoNulls(testCase)
+            filename = 'numeric_datatypes_with_no_nulls.feather';
+            actualTable = featherread(filename);
+
+            variableNames = {'single', ...
+                             'double', ...
+                             'int8', ...
+                             'int16', ...
+                             'int32', ...
+                             'int64', ...
+                             'uint8', ...
+                             'uint16', ...
+                             'uint32', ...
+                             'uint64'};
+            variableTypes = {'single', ...
+                             'double', ...
+                             'int8', ...
+                             'int16', ...
+                             'int32', ...
+                             'int64', ...
+                             'uint8', ...
+                             'uint16', ...
+                             'uint32', ...
+                             'uint64'};
+            numRows = 2;
+            numVariables = 10;
+
+            expectedTable = table('Size', [numRows, numVariables], 'VariableTypes', variableTypes, 'VariableNames', variableNames);
+            expectedTable(1, :) = {1/3, 2/3, 1, 2, 3, 4,  9, 10, 11, 12};
+            expectedTable(2, :) = {4,   5,   5, 6, 7, 8, 13, 14, 15, 16};
+
+            testCase.verifyEqual(actualTable, expectedTable);
+        end
+
+        function NumericDatatypesWithEmptyVariableName(testCase)
+            filename = 'numeric_datatypes_6th_variable_name_is_empty.feather';
+            t = featherread(filename);
+
+            actualVariableName = t.Properties.VariableNames(6);
+            expectedVariableName = {'x'};
+            testCase.verifyEqual(actualVariableName, expectedVariableName);
+        end
+
+        function NumericDatatypesWithNaNRow(testCase)
+            filename = 'numeric_datatypes_with_nan_row.feather';
+            t = featherread(filename);
+
+            actualVariableData = t{3, {'single'}};
+            expectedVariableData = single(NaN);
+            testCase.verifyEqual(actualVariableData, expectedVariableData);
+
+            actualRemainingVariablesData = t{3, {'double','int8','int16','int32','int64',...
+                'uint8','uint16','uint32','uint64'}};
+            expectedRemainingVariablesData = double([NaN, NaN, NaN, NaN, NaN, NaN, NaN, NaN, NaN]);
+            testCase.verifyEqual(actualRemainingVariablesData, expectedRemainingVariablesData);
+        end
+
+        function NumericDatatypesWithNaNColumn(testCase)
+            filename = 'numeric_datatypes_with_nan_column.feather';
+            t = featherread(filename);
+
+            actualVariable6 = t.int64;
+            expectedVariable6 = double([NaN; NaN]);
+            testCase.verifyEqual(actualVariable6, expectedVariable6);
+
+            actualVariable9 = t.uint32;
+            expectedVariable9 = double([NaN;NaN]);
+            testCase.verifyEqual(actualVariable9, expectedVariable9);
+        end
+
+        % %%%%%%%%%%%%%%%%%%%
+        % Negative test cases
+        % %%%%%%%%%%%%%%%%%%%
+        function ErrorIfNotAFeatherFile(testCase)
+            filename = 'not_a_feather_file.feather';
+
+            testCase.verifyError(@() featherread(filename), 'MATLAB:arrow:status:Invalid');
+        end
+
+        function ErrorIfUnableToOpenFile(testCase)
+            filename = 'nonexistent.feather';
+
+            testCase.verifyError(@() featherread(filename), 'MATLAB:arrow:UnableToOpenFile');
+        end
+
+        function ErrorIfCorruptedFeatherFile(testCase)
+            filename = 'corrupted_feather_file.feather';
+
+            testCase.verifyError(@() featherread(filename), 'MATLAB:arrow:status:Invalid');
+        end
+
+        function ErrorIfInvalidFilenameDatatype(testCase)
+            filename = {'numeric_datatypes_with_no_nulls.feather'};
+
+            testCase.verifyError(@() featherread(filename), 'MATLAB:arrow:InvalidFilenameDatatype');
+        end
+
+        function ErroriIfTooManyInputs(testCase)
+            filename = 'numeric_datatypes_with_nan_column.feather';
+
+            testCase.verifyError(@() featherread(filename, 'SomeValue'), 'MATLAB:TooManyInputs');
+        end
+
+        function ErrorIfTooFewInputs(testCase)
+            testCase.verifyError(@() featherread(), 'MATLAB:narginchk:notEnoughInputs');
+        end
+
+    end
+
+end
+