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

[GitHub] [arrow] kevingurney opened a new pull request, #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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

   > **Warning** This pull request is a draft. We will rebase and mark as ready when the changes are finalized.
   
   ### Rationale for this change
   
   Currently, the `arrow.array.<Array>` classes do not support querying the Null values (i.e. validity bitmap) on an Arrow array. Support for encoding Null values is an important part of the Arrow memory format, so the MATLAB Interface to Arrow should support it.
   
   There are likely multiple different APIs that the MATLAB interface should have to support Null values robustly. However, to focus on incremental delivery, we can start by adding a public `Valid` property to the `arrow.array.<Array>` classes, which would return a `logical` array of null values in the given array.
   
   ### What changes are included in this PR?
   
   1. Added a new public property `Valid` to the `arrow.array.Array` superclass.
   2. Implemented basic null value handling for `arrow.array.Float64Array` (i.e. treat `NaN` values in the input MATLAB array as null values in the corresponding `arrow.array.Float64Array`).
   
   Example of creating an `arrow.array.Float64Array` from a MATLAB `double` array containing `NaN` values:
   
   ```matlab
   >> matlabArray = [1, 2, NaN, 4, NaN]'
   
   matlabArray =
   
        1
        2
      NaN
        4
      NaN
   
   >> arrowArray = arrow.array.Float64Array(matlabArray)
   
   arrowArray = 
   
   [
     1,
     2,
     null,
     4,
     null
   ]
   
   >> arrowArray.Valid
   
   ans =
   
     5×1 logical array
   
      1
      1
      0
      1
      0
   
   >> all(~isnan(matlabArray) == arrowArray.Valid)
   
   ans =
   
     logical
   
      1
   ```
   
   ### Are these changes tested?
   
   Yes, we have added the following test points for the `Valid` property of `arrow.array.Float64Array`:
   
   1. `ValidBasic`
   2. `ValidNoNulls`
   4. `ValidAllNulls`
   5. `ValidEmpty`
   
   ### Are there any user-facing changes?
   
   Yes.
   
   There is now a public property `Valid` on the arrow.array.Float64Array` class which is a MATLAB `logical` array encoding the null values in the underlying Arrow array, where `true` indicates an element is valid (i.e. not null) and `false` indicates that an element is invalid (i.e. null).


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

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

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


[GitHub] [arrow] kevingurney commented on a diff in pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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


##########
matlab/src/cpp/arrow/matlab/array/proxy/numeric_array.h:
##########
@@ -26,11 +26,19 @@
 #include "arrow/type_traits.h"
 
 #include "arrow/matlab/array/proxy/array.h"
+#include "arrow/matlab/bit/bit_pack_matlab_logical_array.h"
 
 #include "libmexclass/proxy/Proxy.h"
 
 namespace arrow::matlab::array::proxy {
 
+namespace {
+const uint8_t* getUnpackedValidityBitmap(const ::matlab::data::TypedArray<bool>& valid_elements) {
+    const auto valid_elements_iterator(valid_elements.cbegin());
+    return reinterpret_cast<const uint8_t*>(valid_elements_iterator.operator->());

Review Comment:
   Unfortunately, the current design of the MATLAB Data APIs doesn't provide an easier way of extracting the raw data without making a copy. The `->operator()` syntax is the only way to achieve this at the moment.
   
   We know this isn't a great solution, and we have reached out to the relevant product teams at MathWorks to see if we can come up with a better solution for a future MATLAB release.



-- 
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] ursabot commented on pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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

   ['Python', 'R'] benchmarks have high level of regressions.
   [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/d7f6d9e75931401c91a059b09d4a51f4...f48295651bff45f689c4c73b48e3634e/)
   


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

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

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


[GitHub] [arrow] kevingurney commented on a diff in pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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


##########
matlab/src/cpp/arrow/matlab/array/proxy/array.cc:
##########
@@ -40,4 +43,25 @@ namespace arrow::matlab::array::proxy {
         auto length_mda = factory.createScalar(array->length());
         context.outputs[0] = length_mda;
     }
+
+    void Array::valid(libmexclass::proxy::method::Context& context) {
+        size_t array_length = static_cast<size_t>(array->length());
+        
+        // If the Arrow array has no null values, then return a MATLAB
+        // logical array that is all "true" for the validity bitmap.
+        if (array->null_count() == 0) {
+            ::matlab::data::ArrayFactory factory;
+            auto validity_buffer = factory.createBuffer<bool>(array_length);
+            auto validity_buffer_ptr = validity_buffer.get();
+            std::fill(validity_buffer_ptr, validity_buffer_ptr + array_length, true);
+            ::matlab::data::TypedArray<bool> valid_elements_mda = factory.createArrayFromBuffer<bool>({array_length, 1}, std::move(validity_buffer));
+            context.outputs[0] = valid_elements_mda;
+            return;
+        }
+
+        auto validity_bitmap = array->null_bitmap();
+        ::matlab::data::TypedArray<bool> valid_elements_mda = arrow::matlab::bit::bitUnpackArrowBuffer(validity_bitmap, array_length);

Review Comment:
   Fixed.



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

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

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


[GitHub] [arrow] kou commented on a diff in pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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


##########
matlab/src/cpp/arrow/matlab/array/proxy/array.cc:
##########
@@ -40,4 +43,25 @@ namespace arrow::matlab::array::proxy {
         auto length_mda = factory.createScalar(array->length());
         context.outputs[0] = length_mda;
     }
+
+    void Array::valid(libmexclass::proxy::method::Context& context) {
+        size_t array_length = static_cast<size_t>(array->length());

Review Comment:
   ```suggestion
           auto array_length = static_cast<size_t>(array->length());
   ```



##########
matlab/src/cpp/arrow/matlab/bit/bit_unpack_arrow_buffer.cc:
##########
@@ -0,0 +1,41 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/matlab/bit/bit_unpack_arrow_buffer.h"
+
+#include "arrow/util/bitmap_visit.h"
+
+namespace arrow::matlab::bit {
+    ::matlab::data::TypedArray<bool> bitUnpackArrowBuffer(const std::shared_ptr<arrow::Buffer>& packed_buffer, int64_t length) {
+        const uint8_t* packed_buffer_ptr = packed_buffer->data();
+
+        ::matlab::data::ArrayFactory factory;
+        
+        const size_t array_length = static_cast<size_t>(length);

Review Comment:
   ```suggestion
           const auto array_length = static_cast<size_t>(length);
   ```



##########
matlab/src/cpp/arrow/matlab/bit/bit_unpack_arrow_buffer.cc:
##########
@@ -0,0 +1,41 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/matlab/bit/bit_unpack_arrow_buffer.h"
+
+#include "arrow/util/bitmap_visit.h"
+
+namespace arrow::matlab::bit {
+    ::matlab::data::TypedArray<bool> bitUnpackArrowBuffer(const std::shared_ptr<arrow::Buffer>& packed_buffer, int64_t length) {
+        const uint8_t* packed_buffer_ptr = packed_buffer->data();

Review Comment:
   ```suggestion
           const auto packed_buffer_ptr = packed_buffer->data();
   ```



##########
matlab/src/cpp/arrow/matlab/array/proxy/numeric_array.h:
##########
@@ -26,11 +26,19 @@
 #include "arrow/type_traits.h"
 
 #include "arrow/matlab/array/proxy/array.h"
+#include "arrow/matlab/bit/bit_pack_matlab_logical_array.h"
 
 #include "libmexclass/proxy/Proxy.h"
 
 namespace arrow::matlab::array::proxy {
 
+namespace {
+const uint8_t* getUnpackedValidityBitmap(const ::matlab::data::TypedArray<bool>& valid_elements) {
+    const auto valid_elements_iterator(valid_elements.cbegin());
+    return reinterpret_cast<const uint8_t*>(valid_elements_iterator.operator->());

Review Comment:
   I see.



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

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

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


[GitHub] [arrow] kevingurney commented on a diff in pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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


##########
matlab/src/cpp/arrow/matlab/array/proxy/numeric_array.h:
##########
@@ -26,11 +26,19 @@
 #include "arrow/type_traits.h"
 
 #include "arrow/matlab/array/proxy/array.h"
+#include "arrow/matlab/bit/bit_pack_matlab_logical_array.h"
 
 #include "libmexclass/proxy/Proxy.h"
 
 namespace arrow::matlab::array::proxy {
 
+namespace {
+const uint8_t* getUnpackedValidityBitmap(const ::matlab::data::TypedArray<bool>& valid_elements) {
+    const auto valid_elements_iterator(valid_elements.cbegin());
+    return reinterpret_cast<const uint8_t*>(valid_elements_iterator.operator->());

Review Comment:
   Unfortunately, the current design of the MATLAB Data APIs doesn't provide an easier way of extracting the raw data without making a copy. The `operator->()` syntax is the only way to achieve this at the moment.
   
   We know this isn't a great solution, and we have reached out to the relevant product teams at MathWorks to see if we can come up with a better solution for a future MATLAB release.



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

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

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


[GitHub] [arrow] github-actions[bot] commented on pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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

   * Closes: #35598


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

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

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


[GitHub] [arrow] kou commented on a diff in pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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


##########
matlab/src/cpp/arrow/matlab/array/proxy/array.cc:
##########
@@ -40,4 +43,25 @@ namespace arrow::matlab::array::proxy {
         auto length_mda = factory.createScalar(array->length());
         context.outputs[0] = length_mda;
     }
+
+    void Array::valid(libmexclass::proxy::method::Context& context) {
+        size_t array_length = static_cast<size_t>(array->length());
+        
+        // If the Arrow array has no null values, then return a MATLAB
+        // logical array that is all "true" for the validity bitmap.
+        if (array->null_count() == 0) {
+            ::matlab::data::ArrayFactory factory;
+            auto validity_buffer = factory.createBuffer<bool>(array_length);
+            auto validity_buffer_ptr = validity_buffer.get();
+            std::fill(validity_buffer_ptr, validity_buffer_ptr + array_length, true);
+            ::matlab::data::TypedArray<bool> valid_elements_mda = factory.createArrayFromBuffer<bool>({array_length, 1}, std::move(validity_buffer));
+            context.outputs[0] = valid_elements_mda;
+            return;
+        }
+
+        auto validity_bitmap = array->null_bitmap();
+        ::matlab::data::TypedArray<bool> valid_elements_mda = arrow::matlab::bit::bitUnpackArrowBuffer(validity_bitmap, array_length);

Review Comment:
   Ditto.



##########
matlab/src/cpp/arrow/matlab/array/proxy/numeric_array.h:
##########
@@ -68,12 +81,11 @@ class NumericArray : public arrow::matlab::array::proxy::Array {
                 auto data_buffer = std::make_shared<arrow::Buffer>(reinterpret_cast<const uint8_t*>(dt),
                                                               sizeof(CType) * numeric_mda.getNumberOfElements());
 
-                // TODO: Implement null support
-                std::shared_ptr<arrow::Buffer> null_buffer = nullptr;
+                // Pack the validity bitmap values.
+                auto packed_validity_bitmap = has_validity_bitmap ? arrow::matlab::bit::bitPackMatlabLogicalArray(constructor_arguments[2]).ValueOrDie() : nullptr;

Review Comment:
   In general, we should not call `ValueOrDie()` without checking `ok()` in library. If `!ok()`, an user program is aborted.
   
   Can we raise an exception instead of abort on error?
   (This may be a follow-up task.)



##########
matlab/src/cpp/arrow/matlab/array/proxy/numeric_array.h:
##########
@@ -26,11 +26,19 @@
 #include "arrow/type_traits.h"
 
 #include "arrow/matlab/array/proxy/array.h"
+#include "arrow/matlab/bit/bit_pack_matlab_logical_array.h"
 
 #include "libmexclass/proxy/Proxy.h"
 
 namespace arrow::matlab::array::proxy {
 
+namespace {
+const uint8_t* getUnpackedValidityBitmap(const ::matlab::data::TypedArray<bool>& valid_elements) {
+    const auto valid_elements_iterator(valid_elements.cbegin());
+    return reinterpret_cast<const uint8_t*>(valid_elements_iterator.operator->());

Review Comment:
   Wow!
   
   `::matlab::data::TypedArray` doesn't provide a method that returns it's raw data, right?



##########
matlab/src/cpp/arrow/matlab/array/proxy/array.cc:
##########
@@ -40,4 +43,25 @@ namespace arrow::matlab::array::proxy {
         auto length_mda = factory.createScalar(array->length());
         context.outputs[0] = length_mda;
     }
+
+    void Array::valid(libmexclass::proxy::method::Context& context) {
+        size_t array_length = static_cast<size_t>(array->length());
+        
+        // If the Arrow array has no null values, then return a MATLAB
+        // logical array that is all "true" for the validity bitmap.
+        if (array->null_count() == 0) {
+            ::matlab::data::ArrayFactory factory;
+            auto validity_buffer = factory.createBuffer<bool>(array_length);
+            auto validity_buffer_ptr = validity_buffer.get();
+            std::fill(validity_buffer_ptr, validity_buffer_ptr + array_length, true);
+            ::matlab::data::TypedArray<bool> valid_elements_mda = factory.createArrayFromBuffer<bool>({array_length, 1}, std::move(validity_buffer));

Review Comment:
   Can we use `auto` here?
   
   ```suggestion
               auto valid_elements_mda = factory.createArrayFromBuffer<bool>({array_length, 1}, std::move(validity_buffer));
   ```



##########
matlab/src/cpp/arrow/matlab/bit/bit_pack_matlab_logical_array.cc:
##########
@@ -0,0 +1,64 @@
+// 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 <cmath> // std::ceil
+
+#include <arrow/util/bit_util.h>
+#include <arrow/util/bitmap_generate.h>
+
+#include "arrow/matlab/bit/bit_pack_matlab_logical_array.h"
+
+namespace arrow::matlab::bit {
+
+    // Calculate the number of bytes required in the bit-packed validity buffer.
+    int64_t bitPackedLength(int64_t num_elements) {
+        // Since MATLAB logical values are encoded using a full byte (8 bits),
+        // we can divide the number of elements in the logical array by 8 to get
+        // the bit packed length.
+        return static_cast<int64_t>(std::ceil(num_elements / 8.0));
+    }
+
+    // Pack an unpacked MATLAB logical array into into a bit-packed arrow::Buffer.
+    arrow::Result<std::shared_ptr<arrow::Buffer>> bitPackMatlabLogicalArray(const ::matlab::data::TypedArray<bool> matlab_logical_array) {
+        // Validate that the input arrow::Buffer has sufficient size to store a full bit-packed
+        // representation of the input MATLAB logical array.
+        const auto unpacked_buffer_length = matlab_logical_array.getNumberOfElements();
+
+        // Compute the bit packed length from the unpacked length.
+        const auto packed_buffer_length = bitPackedLength(unpacked_buffer_length);
+
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<arrow::ResizableBuffer> packed_validity_bitmap_buffer,  arrow::AllocateResizableBuffer(packed_buffer_length));
+
+        // Get pointers to the internal uint8_t arrays behind arrow::Buffer and mxArray
+        // Get raw bool array pointer from MATLAB logical array.
+        // Get an iterator to the raw bool data behind the MATLAB logical array.
+        auto unpacked_bool_data_iterator = matlab_logical_array.cbegin();
+
+        // Iterate over the mxLogical array and write bit-packed bools to the arrow::Buffer.
+        // Call into a loop-unrolled Arrow utility for better performance when bit-packing.
+        auto generator = [&]() -> bool { return *(unpacked_bool_data_iterator++); };
+        const int64_t start_offset = 0;
+
+        auto mutable_data = packed_validity_bitmap_buffer->mutable_data();
+
+        arrow::internal::GenerateBitsUnrolled(mutable_data, start_offset, unpacked_buffer_length, generator);
+
+        auto buffer_result = std::static_pointer_cast<arrow::Buffer>(packed_validity_bitmap_buffer);
+        return arrow::Result<std::shared_ptr<arrow::Buffer>>{buffer_result};

Review Comment:
   Can we simplify this?
   
   * `std::shared_ptr<arrow::ResizableBuffer> packed_validity_bitmap_buffer` -> `auto packed_validity_bitmap_buffer`
   *  `auto buffer_result = std::static_pointer_cast<arrow::Buffer>(packed_validity_bitmap_buffer); return arrow::Result<std::shared_ptr<arrow::Buffer>>{buffer_result};` -> `return packed_validity_bitmap_buffer`
   
   ```suggestion
           ARROW_ASSIGN_OR_RAISE(auto packed_validity_bitmap_buffer,  arrow::AllocateResizableBuffer(packed_buffer_length));
   
           // Get pointers to the internal uint8_t arrays behind arrow::Buffer and mxArray
           // Get raw bool array pointer from MATLAB logical array.
           // Get an iterator to the raw bool data behind the MATLAB logical array.
           auto unpacked_bool_data_iterator = matlab_logical_array.cbegin();
   
           // Iterate over the mxLogical array and write bit-packed bools to the arrow::Buffer.
           // Call into a loop-unrolled Arrow utility for better performance when bit-packing.
           auto generator = [&]() -> bool { return *(unpacked_bool_data_iterator++); };
           const int64_t start_offset = 0;
   
           auto mutable_data = packed_validity_bitmap_buffer->mutable_data();
   
           arrow::internal::GenerateBitsUnrolled(mutable_data, start_offset, unpacked_buffer_length, generator);
   
           return packed_validity_bitmap_buffer;
   ```



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

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

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


[GitHub] [arrow] kevingurney commented on a diff in pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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


##########
matlab/src/cpp/arrow/matlab/array/proxy/array.cc:
##########
@@ -40,4 +43,25 @@ namespace arrow::matlab::array::proxy {
         auto length_mda = factory.createScalar(array->length());
         context.outputs[0] = length_mda;
     }
+
+    void Array::valid(libmexclass::proxy::method::Context& context) {
+        size_t array_length = static_cast<size_t>(array->length());
+        
+        // If the Arrow array has no null values, then return a MATLAB
+        // logical array that is all "true" for the validity bitmap.
+        if (array->null_count() == 0) {
+            ::matlab::data::ArrayFactory factory;
+            auto validity_buffer = factory.createBuffer<bool>(array_length);
+            auto validity_buffer_ptr = validity_buffer.get();
+            std::fill(validity_buffer_ptr, validity_buffer_ptr + array_length, true);
+            ::matlab::data::TypedArray<bool> valid_elements_mda = factory.createArrayFromBuffer<bool>({array_length, 1}, std::move(validity_buffer));

Review Comment:
   Fixed.



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

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

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


[GitHub] [arrow] assignUser commented on pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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

   I don't feel comfortable "officially" reviewing these changes but the reasoning is sound and the quality of the PR description is great. It has been nice to see the constant stream of recent contributions to the matlab component! :tada: 


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

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

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


[GitHub] [arrow] kevingurney commented on a diff in pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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


##########
matlab/src/cpp/arrow/matlab/array/proxy/numeric_array.h:
##########
@@ -68,12 +81,11 @@ class NumericArray : public arrow::matlab::array::proxy::Array {
                 auto data_buffer = std::make_shared<arrow::Buffer>(reinterpret_cast<const uint8_t*>(dt),
                                                               sizeof(CType) * numeric_mda.getNumberOfElements());
 
-                // TODO: Implement null support
-                std::shared_ptr<arrow::Buffer> null_buffer = nullptr;
+                // Pack the validity bitmap values.
+                auto packed_validity_bitmap = has_validity_bitmap ? arrow::matlab::bit::bitPackMatlabLogicalArray(constructor_arguments[2]).ValueOrDie() : nullptr;

Review Comment:
   That's a very good point!
   
   We need to add a bit more infrastructure to mathworks/libmexclass to make exception handling more straightforward. So, it might be easier if we follow up with a separate pull request to handle this after updating libmexclass.
   
   We have been exploring a few different approaches to error handling in libmexclass, but we haven't decided which one makes the most sense yet. We know this is something that needs to be prioritized in the near term.
   
   I've captured this in https://github.com/apache/arrow/issues/35768.



-- 
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 merged pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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


-- 
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] ursabot commented on pull request #35655: GH-35598: [MATLAB] Add a public `Valid` property to to the `MATLAB arrow.array.` classes to query Null values (i.e. validity bitmap support)

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

   Benchmark runs are scheduled for baseline = d14b42ac6b25823042b9da83ee16615579a81fa1 and contender = 05fe0d25834fd1629d71ceb51f0281b44a511f94. 05fe0d25834fd1629d71ceb51f0281b44a511f94 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/51122813cdce4e9d8476b4217a670e33...9c29e610b30a47549a463da71c2a9ac7/)
   [Failed :arrow_down:0.17% :arrow_up:0.0%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/6b9bb1a551d84a62a27ec5f453bf5aef...67739be6ef18463f80a1a0c4c1424b53/)
   [Failed :arrow_down:12.82% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/d7f6d9e75931401c91a059b09d4a51f4...f48295651bff45f689c4c73b48e3634e/)
   [Failed :arrow_down:0.0% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/97f887779c4e4f42835282d76931e3ff...bb7934ce1d9548ae9546899ec72a72d2/)
   Buildkite builds:
   [Finished] [`05fe0d25` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2944)
   [Failed] [`05fe0d25` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2980)
   [Failed] [`05fe0d25` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2945)
   [Failed] [`05fe0d25` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2970)
   [Finished] [`d14b42ac` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/2943)
   [Failed] [`d14b42ac` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/2979)
   [Finished] [`d14b42ac` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/2944)
   [Failed] [`d14b42ac` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/2969)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


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