You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by GitBox <gi...@apache.org> on 2022/08/03 19:24:48 UTC

[GitHub] [arrow-nanoarrow] lidavidm commented on a diff in pull request #10: Implement bitmap setters, getters, and element-wise builder

lidavidm commented on code in PR #10:
URL: https://github.com/apache/arrow-nanoarrow/pull/10#discussion_r937046260


##########
src/nanoarrow/typedefs_inline.h:
##########
@@ -0,0 +1,172 @@
+// 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 NANOARROW_TYPEDEFS_INLINE_H_INCLUDED
+#define NANOARROW_TYPEDEFS_INLINE_H_INCLUDED
+
+#include <stdint.h>
+
+/// \defgroup nanoarrow-inline-typedef Type definitions used in inlined implementations

Review Comment:
   This still needs the `extern "C"`



##########
src/nanoarrow/bitmap_inline.h:
##########
@@ -0,0 +1,131 @@
+// 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 NANOARROW_BITMAP_INLINE_H_INCLUDED
+#define NANOARROW_BITMAP_INLINE_H_INCLUDED
+
+#include <stdlib.h>
+#include <string.h>
+
+#include "buffer_inline.h"
+#include "typedefs_inline.h"
+
+static inline int8_t ArrowBitmapElement(const void* bitmap, int64_t i) {

Review Comment:
   It may be worth stealing Arrow's implementations of these utilities: https://github.com/apache/arrow/blob/3b987d92d14ce7b5f5ccd2afb7366273e20348d4/cpp/src/arrow/util/bit_util.h#L296



##########
src/nanoarrow/bitmap_inline.h:
##########
@@ -0,0 +1,131 @@
+// 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 NANOARROW_BITMAP_INLINE_H_INCLUDED
+#define NANOARROW_BITMAP_INLINE_H_INCLUDED
+
+#include <stdlib.h>
+#include <string.h>
+
+#include "buffer_inline.h"
+#include "typedefs_inline.h"
+
+static inline int8_t ArrowBitmapElement(const void* bitmap, int64_t i) {
+  const int8_t* bitmap_char = (const int8_t*)bitmap;
+  return 0 != (bitmap_char[i / 8] & ((int8_t)0x01) << (i % 8));
+}
+
+static inline void ArrowBitmapSetElement(void* bitmap, int64_t i, int8_t value) {
+  int8_t* bitmap_char = (int8_t*)bitmap;
+  int8_t mask = 0x01 << (i % 8);
+  if (value) {
+    bitmap_char[i / 8] |= mask;
+  } else {
+    bitmap_char[i / 8] &= ~mask;
+  }
+}
+
+static inline int64_t ArrowBitmapCountTrue(const void* bitmap, int64_t i_from,
+                                           int64_t i_to) {
+  int64_t count = 0;
+  for (int64_t i = i_from; i < i_to; i++) {
+    count += ArrowBitmapElement(bitmap, i);

Review Comment:
   I wonder if this optimized to popcnt or not…but Arrow also has implementations to steal!



##########
src/nanoarrow/nanoarrow.c:
##########
@@ -17,6 +17,7 @@
 
 #include "allocator.c"
 #include "buffer.c"
+#include "bitmap.c"

Review Comment:
   Does this file exist?



##########
src/nanoarrow/nanoarrow.h:
##########
@@ -483,82 +372,117 @@ ArrowErrorCode ArrowSchemaViewInit(struct ArrowSchemaView* schema_view,
 
 /// }@
 
-/// \defgroup nanoarrow-buffer-builder Growable buffer builders
-
-/// \brief An owning mutable view of a buffer
-struct ArrowBuffer {
-  /// \brief A pointer to the start of the buffer
-  ///
-  /// If capacity_bytes is 0, this value may be NULL.
-  uint8_t* data;
-
-  /// \brief The size of the buffer in bytes
-  int64_t size_bytes;
-
-  /// \brief The capacity of the buffer in bytes
-  int64_t capacity_bytes;
-
-  /// \brief The allocator that will be used to reallocate and/or free the buffer
-  struct ArrowBufferAllocator* allocator;
-};
+/// \defgroup nanoarrow-buffer Owning, growable buffers
 
 /// \brief Initialize an ArrowBuffer
 ///
 /// Initialize a buffer with a NULL, zero-size buffer using the default
 /// buffer allocator.
-void ArrowBufferInit(struct ArrowBuffer* buffer);
+static inline void ArrowBufferInit(struct ArrowBuffer* buffer);
 
 /// \brief Set a newly-initialized buffer's allocator
 ///
 /// Returns EINVAL if the buffer has already been allocated.
-ArrowErrorCode ArrowBufferSetAllocator(struct ArrowBuffer* buffer,
-                                       struct ArrowBufferAllocator* allocator);
+static inline ArrowErrorCode ArrowBufferSetAllocator(
+    struct ArrowBuffer* buffer, struct ArrowBufferAllocator* allocator);
 
 /// \brief Reset an ArrowBuffer
 ///
 /// Releases the buffer using the allocator's free method if
 /// the buffer's data member is non-null, sets the data member
 /// to NULL, and sets the buffer's size and capacity to 0.
-void ArrowBufferReset(struct ArrowBuffer* buffer);
+static inline void ArrowBufferReset(struct ArrowBuffer* buffer);
 
 /// \brief Move an ArrowBuffer
 ///
 /// Transfers the buffer data and lifecycle management to another
 /// address and resets buffer.
-void ArrowBufferMove(struct ArrowBuffer* buffer, struct ArrowBuffer* buffer_out);
+static inline void ArrowBufferMove(struct ArrowBuffer* buffer,
+                                   struct ArrowBuffer* buffer_out);
 
 /// \brief Grow or shrink a buffer to a given capacity
 ///
 /// When shrinking the capacity of the buffer, the buffer is only reallocated
 /// if shrink_to_fit is non-zero. Calling ArrowBufferResize() does not
 /// adjust the buffer's size member except to ensure that the invariant
 /// capacity >= size remains true.
-ArrowErrorCode ArrowBufferResize(struct ArrowBuffer* buffer, int64_t new_capacity_bytes,
-                                 char shrink_to_fit);
+static inline ArrowErrorCode ArrowBufferResize(struct ArrowBuffer* buffer,
+                                               int64_t new_capacity_bytes,
+                                               char shrink_to_fit);
 
 /// \brief Ensure a buffer has at least a given additional capacity
 ///
 /// Ensures that the buffer has space to append at least
 /// additional_size_bytes, overallocating when required.
-ArrowErrorCode ArrowBufferReserve(struct ArrowBuffer* buffer,
-                                  int64_t additional_size_bytes);
+static inline ArrowErrorCode ArrowBufferReserve(struct ArrowBuffer* buffer,
+                                                int64_t additional_size_bytes);
 
 /// \brief Write data to buffer and increment the buffer size
 ///
 /// This function does not check that buffer has the required capacity
-void ArrowBufferAppendUnsafe(struct ArrowBuffer* buffer, const void* data,
-                             int64_t size_bytes);
+static inline void ArrowBufferAppendUnsafe(struct ArrowBuffer* buffer, const void* data,
+                                           int64_t size_bytes);
 
 /// \brief Write data to buffer and increment the buffer size
 ///
 /// This function writes and ensures that the buffer has the required capacity,
 /// possibly by reallocating the buffer. Like ArrowBufferReserve, this will
 /// overallocate when reallocation is required.
-ArrowErrorCode ArrowBufferAppend(struct ArrowBuffer* buffer, const void* data,
-                                 int64_t size_bytes);
+static inline ArrowErrorCode ArrowBufferAppend(struct ArrowBuffer* buffer,
+                                               const void* data, int64_t size_bytes);
+
+/// }@
+
+/// \defgroup nanoarrow-bitmap Bitmap utilities
+
+/// \brief Extract a boolean value from a bitmap
+static inline int8_t ArrowBitmapElement(const void* bitmap, int64_t i);

Review Comment:
   nit: `ArrowBitmapGet`/Set/CountSet/CountUnset/etc.?



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

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