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 2016/03/05 00:01:44 UTC

arrow git commit: ARROW-23: Add a logical Column data structure

Repository: arrow
Updated Branches:
  refs/heads/master 3b777c7f4 -> 9c2b95446


ARROW-23: Add a logical Column data structure

I also added global const instances of common primitive types

Author: Wes McKinney <we...@apache.org>

Closes #15 from wesm/ARROW-23 and squashes the following commits:

1835d33 [Wes McKinney] Don't use auto
988135c [Wes McKinney] Add Column chunk type validation function
8a2e40e [Wes McKinney] Remove unneeded operator()/shared_from_this experiment
de9ec70 [Wes McKinney] Aggregate null counts too
7049314 [Wes McKinney] cpplint
a565d26 [Wes McKinney] Add ChunkedArray / Column ctors, test passes
0648ed2 [Wes McKinney] Prototyping


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/9c2b9544
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/9c2b9544
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/9c2b9544

Branch: refs/heads/master
Commit: 9c2b95446abe1ec4dd5c25215c9595a3d7b49f2b
Parents: 3b777c7
Author: Wes McKinney <we...@apache.org>
Authored: Fri Mar 4 15:02:10 2016 -0800
Committer: Wes McKinney <we...@apache.org>
Committed: Fri Mar 4 15:02:10 2016 -0800

----------------------------------------------------------------------
 cpp/CMakeLists.txt                 |   2 +
 cpp/src/arrow/array.h              |   1 -
 cpp/src/arrow/schema-test.cc       |   7 +--
 cpp/src/arrow/table/CMakeLists.txt |  39 ++++++++++++
 cpp/src/arrow/table/column-test.cc |  93 ++++++++++++++++++++++++++++
 cpp/src/arrow/table/column.cc      |  62 +++++++++++++++++++
 cpp/src/arrow/table/column.h       | 103 ++++++++++++++++++++++++++++++++
 cpp/src/arrow/type.cc              |  12 ++++
 cpp/src/arrow/type.h               |  17 ++++++
 cpp/src/arrow/types/list.h         |   2 +-
 cpp/src/arrow/types/primitive.h    |  20 ++++---
 cpp/src/arrow/util/bit-util.h      |   4 ++
 12 files changed, 347 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index f425c5f..15afb1a 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -458,10 +458,12 @@ endif()
 
 add_subdirectory(src/arrow)
 add_subdirectory(src/arrow/util)
+add_subdirectory(src/arrow/table)
 add_subdirectory(src/arrow/types)
 
 set(LINK_LIBS
   arrow_util
+  arrow_table
   arrow_types)
 
 set(ARROW_SRCS

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/array.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index 0632146..85e853e 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -81,7 +81,6 @@ class Array {
   DISALLOW_COPY_AND_ASSIGN(Array);
 };
 
-
 typedef std::shared_ptr<Array> ArrayPtr;
 
 } // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/schema-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/schema-test.cc b/cpp/src/arrow/schema-test.cc
index 3debb9c..7c190d0 100644
--- a/cpp/src/arrow/schema-test.cc
+++ b/cpp/src/arrow/schema-test.cc
@@ -31,7 +31,7 @@ using std::vector;
 namespace arrow {
 
 TEST(TestField, Basics) {
-  shared_ptr<DataType> ftype = std::make_shared<Int32Type>();
+  shared_ptr<DataType> ftype = INT32;
   shared_ptr<DataType> ftype_nn = std::make_shared<Int32Type>(false);
   Field f0("f0", ftype);
   Field f0_nn("f0", ftype_nn);
@@ -44,7 +44,7 @@ TEST(TestField, Basics) {
 }
 
 TEST(TestField, Equals) {
-  shared_ptr<DataType> ftype = std::make_shared<Int32Type>();
+  shared_ptr<DataType> ftype = INT32;
   shared_ptr<DataType> ftype_nn = std::make_shared<Int32Type>(false);
 
   Field f0("f0", ftype);
@@ -61,8 +61,7 @@ class TestSchema : public ::testing::Test {
 };
 
 TEST_F(TestSchema, Basics) {
-  auto f0 = std::make_shared<Field>("f0", std::make_shared<Int32Type>());
-
+  auto f0 = std::make_shared<Field>("f0", INT32);
   auto f1 = std::make_shared<Field>("f1", std::make_shared<UInt8Type>(false));
   auto f1_optional = std::make_shared<Field>("f1", std::make_shared<UInt8Type>());
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/table/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/CMakeLists.txt b/cpp/src/arrow/table/CMakeLists.txt
new file mode 100644
index 0000000..a401622
--- /dev/null
+++ b/cpp/src/arrow/table/CMakeLists.txt
@@ -0,0 +1,39 @@
+# 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.
+
+#######################################
+# arrow_table
+#######################################
+
+set(TABLE_SRCS
+  column.cc
+)
+
+set(TABLE_LIBS
+)
+
+add_library(arrow_table STATIC
+  ${TABLE_SRCS}
+)
+target_link_libraries(arrow_table ${TABLE_LIBS})
+SET_TARGET_PROPERTIES(arrow_table PROPERTIES LINKER_LANGUAGE CXX)
+
+# Headers: top level
+install(FILES
+  DESTINATION include/arrow/table)
+
+ADD_ARROW_TEST(column-test)

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/table/column-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/column-test.cc b/cpp/src/arrow/table/column-test.cc
new file mode 100644
index 0000000..15f554f
--- /dev/null
+++ b/cpp/src/arrow/table/column-test.cc
@@ -0,0 +1,93 @@
+// 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 <gtest/gtest.h>
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/field.h"
+#include "arrow/schema.h"
+#include "arrow/table/column.h"
+#include "arrow/test-util.h"
+#include "arrow/type.h"
+#include "arrow/types/integer.h"
+#include "arrow/util/bit-util.h"
+#include "arrow/util/buffer.h"
+#include "arrow/util/memory-pool.h"
+#include "arrow/util/status.h"
+
+using std::shared_ptr;
+using std::vector;
+
+namespace arrow {
+
+class TestColumn : public ::testing::Test {
+ public:
+  void SetUp() {
+    pool_ = GetDefaultMemoryPool();
+  }
+
+  template <typename ArrayType>
+  std::shared_ptr<Array> MakeArray(int32_t length, int32_t null_count = 0) {
+    auto data = std::make_shared<PoolBuffer>(pool_);
+    auto nulls = std::make_shared<PoolBuffer>(pool_);
+    data->Resize(length * sizeof(typename ArrayType::value_type));
+    nulls->Resize(util::bytes_for_bits(length));
+    return std::make_shared<ArrayType>(length, data, 10, nulls);
+  }
+
+ protected:
+  MemoryPool* pool_;
+
+  std::shared_ptr<ChunkedArray> data_;
+  std::unique_ptr<Column> column_;
+};
+
+TEST_F(TestColumn, BasicAPI) {
+  ArrayVector arrays;
+  arrays.push_back(MakeArray<Int32Array>(100));
+  arrays.push_back(MakeArray<Int32Array>(100, 10));
+  arrays.push_back(MakeArray<Int32Array>(100, 20));
+
+  auto field = std::make_shared<Field>("c0", INT32);
+  column_.reset(new Column(field, arrays));
+
+  ASSERT_EQ("c0", column_->name());
+  ASSERT_TRUE(column_->type()->Equals(INT32));
+  ASSERT_EQ(300, column_->length());
+  ASSERT_EQ(30, column_->null_count());
+  ASSERT_EQ(3, column_->data()->num_chunks());
+}
+
+TEST_F(TestColumn, ChunksInhomogeneous) {
+  ArrayVector arrays;
+  arrays.push_back(MakeArray<Int32Array>(100));
+  arrays.push_back(MakeArray<Int32Array>(100, 10));
+
+  auto field = std::make_shared<Field>("c0", INT32);
+  column_.reset(new Column(field, arrays));
+
+  ASSERT_OK(column_->ValidateData());
+
+  arrays.push_back(MakeArray<Int16Array>(100, 10));
+  column_.reset(new Column(field, arrays));
+  ASSERT_RAISES(Invalid, column_->ValidateData());
+}
+
+} // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/table/column.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/column.cc b/cpp/src/arrow/table/column.cc
new file mode 100644
index 0000000..82750cf
--- /dev/null
+++ b/cpp/src/arrow/table/column.cc
@@ -0,0 +1,62 @@
+// 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/table/column.h"
+
+#include <memory>
+#include <sstream>
+
+#include "arrow/field.h"
+#include "arrow/util/status.h"
+
+namespace arrow {
+
+ChunkedArray::ChunkedArray(const ArrayVector& chunks) :
+    chunks_(chunks) {
+  length_ = 0;
+  for (const std::shared_ptr<Array>& chunk : chunks) {
+    length_ += chunk->length();
+    null_count_ += chunk->null_count();
+  }
+}
+
+Column::Column(const std::shared_ptr<Field>& field, const ArrayVector& chunks) :
+    field_(field) {
+  data_ = std::make_shared<ChunkedArray>(chunks);
+}
+
+Column::Column(const std::shared_ptr<Field>& field,
+    const std::shared_ptr<ChunkedArray>& data) :
+    field_(field),
+    data_(data) {}
+
+Status Column::ValidateData() {
+  for (int i = 0; i < data_->num_chunks(); ++i) {
+    const std::shared_ptr<DataType>& type = data_->chunk(i)->type();
+    if (!this->type()->Equals(type)) {
+      std::stringstream ss;
+      ss << "In chunk " << i << " expected type "
+         << this->type()->ToString()
+         << " but saw "
+         << type->ToString();
+      return Status::Invalid(ss.str());
+    }
+  }
+  return Status::OK();
+}
+
+} // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/table/column.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/table/column.h b/cpp/src/arrow/table/column.h
new file mode 100644
index 0000000..9e9064e
--- /dev/null
+++ b/cpp/src/arrow/table/column.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 ARROW_TABLE_COLUMN_H
+#define ARROW_TABLE_COLUMN_H
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/field.h"
+
+namespace arrow {
+
+typedef std::vector<std::shared_ptr<Array> > ArrayVector;
+
+// A data structure managing a list of primitive Arrow arrays logically as one
+// large array
+class ChunkedArray {
+ public:
+  explicit ChunkedArray(const ArrayVector& chunks);
+
+  // @returns: the total length of the chunked array; computed on construction
+  int64_t length() const {
+    return length_;
+  }
+
+  int64_t null_count() const {
+    return null_count_;
+  }
+
+  int num_chunks() const {
+    return chunks_.size();
+  }
+
+  const std::shared_ptr<Array>& chunk(int i) const {
+    return chunks_[i];
+  }
+
+ protected:
+  ArrayVector chunks_;
+  int64_t length_;
+  int64_t null_count_;
+};
+
+// An immutable column data structure consisting of a field (type metadata) and
+// a logical chunked data array (which can be validated as all being the same
+// type).
+class Column {
+ public:
+  Column(const std::shared_ptr<Field>& field, const ArrayVector& chunks);
+  Column(const std::shared_ptr<Field>& field,
+      const std::shared_ptr<ChunkedArray>& data);
+
+  int64_t length() const {
+    return data_->length();
+  }
+
+  int64_t null_count() const {
+    return data_->null_count();
+  }
+
+  // @returns: the column's name in the passed metadata
+  const std::string& name() const {
+    return field_->name;
+  }
+
+  // @returns: the column's type according to the metadata
+  const std::shared_ptr<DataType>& type() const {
+    return field_->type;
+  }
+
+  // @returns: the column's data as a chunked logical array
+  const std::shared_ptr<ChunkedArray>& data() const {
+    return data_;
+  }
+  // Verify that the column's array data is consistent with the passed field's
+  // metadata
+  Status ValidateData();
+
+ protected:
+  std::shared_ptr<Field> field_;
+  std::shared_ptr<ChunkedArray> data_;
+};
+
+} // namespace arrow
+
+#endif  // ARROW_TABLE_COLUMN_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/type.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index 492eee5..ff145e2 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -19,4 +19,16 @@
 
 namespace arrow {
 
+const std::shared_ptr<BooleanType> BOOL = std::make_shared<BooleanType>();
+const std::shared_ptr<UInt8Type> UINT8 = std::make_shared<UInt8Type>();
+const std::shared_ptr<UInt16Type> UINT16 = std::make_shared<UInt16Type>();
+const std::shared_ptr<UInt32Type> UINT32 = std::make_shared<UInt32Type>();
+const std::shared_ptr<UInt64Type> UINT64 = std::make_shared<UInt64Type>();
+const std::shared_ptr<Int8Type> INT8 = std::make_shared<Int8Type>();
+const std::shared_ptr<Int16Type> INT16 = std::make_shared<Int16Type>();
+const std::shared_ptr<Int32Type> INT32 = std::make_shared<Int32Type>();
+const std::shared_ptr<Int64Type> INT64 = std::make_shared<Int64Type>();
+const std::shared_ptr<FloatType> FLOAT = std::make_shared<FloatType>();
+const std::shared_ptr<DoubleType> DOUBLE = std::make_shared<DoubleType>();
+
 } // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index 04cdb52..4193a0e 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -142,10 +142,15 @@ struct DataType {
       nullable(nullable) {}
 
   virtual bool Equals(const DataType* other) {
+    // Call with a pointer so more friendly to subclasses
     return this == other || (this->type == other->type &&
         this->nullable == other->nullable);
   }
 
+  bool Equals(const std::shared_ptr<DataType>& other) {
+    return Equals(other.get());
+  }
+
   virtual std::string ToString() const = 0;
 };
 
@@ -244,6 +249,18 @@ struct DoubleType : public PrimitiveType<DoubleType> {
   PRIMITIVE_DECL(DoubleType, double, DOUBLE, 8, "double");
 };
 
+extern const std::shared_ptr<BooleanType> BOOL;
+extern const std::shared_ptr<UInt8Type> UINT8;
+extern const std::shared_ptr<UInt16Type> UINT16;
+extern const std::shared_ptr<UInt32Type> UINT32;
+extern const std::shared_ptr<UInt64Type> UINT64;
+extern const std::shared_ptr<Int8Type> INT8;
+extern const std::shared_ptr<Int16Type> INT16;
+extern const std::shared_ptr<Int32Type> INT32;
+extern const std::shared_ptr<Int64Type> INT64;
+extern const std::shared_ptr<FloatType> FLOAT;
+extern const std::shared_ptr<DoubleType> DOUBLE;
+
 } // namespace arrow
 
 #endif  // ARROW_TYPE_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/types/list.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/types/list.h b/cpp/src/arrow/types/list.h
index 1fc8353..f39fe5c 100644
--- a/cpp/src/arrow/types/list.h
+++ b/cpp/src/arrow/types/list.h
@@ -132,7 +132,7 @@ class ListBuilder : public Int32Builder {
   //
   // If passed, null_bytes is of equal length to values, and any nonzero byte
   // will be considered as a null for that slot
-  Status Append(T* values, int32_t length, uint8_t* null_bytes = nullptr) {
+  Status Append(value_type* values, int32_t length, uint8_t* null_bytes = nullptr) {
     if (length_ + length > capacity_) {
       int32_t new_capacity = util::next_power2(length_ + length);
       RETURN_NOT_OK(Resize(new_capacity));

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/types/primitive.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/types/primitive.h b/cpp/src/arrow/types/primitive.h
index 49040fb..09d43e7 100644
--- a/cpp/src/arrow/types/primitive.h
+++ b/cpp/src/arrow/types/primitive.h
@@ -60,7 +60,7 @@ class PrimitiveArray : public Array {
 template <typename TypeClass>
 class PrimitiveArrayImpl : public PrimitiveArray {
  public:
-  typedef typename TypeClass::c_type T;
+  typedef typename TypeClass::c_type value_type;
 
   PrimitiveArrayImpl() : PrimitiveArray() {}
 
@@ -81,9 +81,11 @@ class PrimitiveArrayImpl : public PrimitiveArray {
     return PrimitiveArray::Equals(*static_cast<const PrimitiveArray*>(&other));
   }
 
-  const T* raw_data() const { return reinterpret_cast<const T*>(raw_data_);}
+  const value_type* raw_data() const {
+    return reinterpret_cast<const value_type*>(raw_data_);
+  }
 
-  T Value(int i) const {
+  value_type Value(int i) const {
     return raw_data()[i];
   }
 
@@ -96,12 +98,12 @@ class PrimitiveArrayImpl : public PrimitiveArray {
 template <typename Type, typename ArrayType>
 class PrimitiveBuilder : public ArrayBuilder {
  public:
-  typedef typename Type::c_type T;
+  typedef typename Type::c_type value_type;
 
   explicit PrimitiveBuilder(MemoryPool* pool, const TypePtr& type) :
       ArrayBuilder(pool, type),
       values_(nullptr) {
-    elsize_ = sizeof(T);
+    elsize_ = sizeof(value_type);
   }
 
   virtual ~PrimitiveBuilder() {}
@@ -141,7 +143,7 @@ class PrimitiveBuilder : public ArrayBuilder {
   }
 
   // Scalar append
-  Status Append(T val, bool is_null = false) {
+  Status Append(value_type val, bool is_null = false) {
     if (length_ == capacity_) {
       // If the capacity was not already a multiple of 2, do so here
       RETURN_NOT_OK(Resize(util::next_power2(capacity_ + 1)));
@@ -158,7 +160,7 @@ class PrimitiveBuilder : public ArrayBuilder {
   //
   // If passed, null_bytes is of equal length to values, and any nonzero byte
   // will be considered as a null for that slot
-  Status Append(const T* values, int32_t length,
+  Status Append(const value_type* values, int32_t length,
       const uint8_t* null_bytes = nullptr) {
     if (length_ + length > capacity_) {
       int32_t new_capacity = util::next_power2(length_ + length);
@@ -215,8 +217,8 @@ class PrimitiveBuilder : public ArrayBuilder {
     return Status::OK();
   }
 
-  T* raw_buffer() {
-    return reinterpret_cast<T*>(values_->mutable_data());
+  value_type* raw_buffer() {
+    return reinterpret_cast<value_type*>(values_->mutable_data());
   }
 
   std::shared_ptr<Buffer> buffer() const {

http://git-wip-us.apache.org/repos/asf/arrow/blob/9c2b9544/cpp/src/arrow/util/bit-util.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/bit-util.h b/cpp/src/arrow/util/bit-util.h
index 841f617..5e7197f 100644
--- a/cpp/src/arrow/util/bit-util.h
+++ b/cpp/src/arrow/util/bit-util.h
@@ -33,6 +33,10 @@ static inline int64_t ceil_byte(int64_t size) {
   return (size + 7) & ~7;
 }
 
+static inline int64_t bytes_for_bits(int64_t size) {
+  return ceil_byte(size) / 8;
+}
+
 static inline int64_t ceil_2bytes(int64_t size) {
   return (size + 15) & ~15;
 }