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 2017/03/16 16:02:01 UTC

arrow git commit: ARROW-520: [C++] STL-compliant allocator

Repository: arrow
Updated Branches:
  refs/heads/master c8d15d467 -> 3b650014f


ARROW-520: [C++] STL-compliant allocator

Ready for review

Author: Johan Mabille <jo...@gmail.com>

Closes #381 from JohanMabille/stl_allocator and squashes the following commits:

53c6821 [Johan Mabille] stl allocator


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

Branch: refs/heads/master
Commit: 3b650014f6c59c6cf6f488572c5cd340bf2da453
Parents: c8d15d4
Author: Johan Mabille <jo...@gmail.com>
Authored: Thu Mar 16 12:01:13 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Thu Mar 16 12:01:13 2017 -0400

----------------------------------------------------------------------
 cpp/src/arrow/CMakeLists.txt    |  2 +
 cpp/src/arrow/allocator-test.cc | 72 ++++++++++++++++++++++++++
 cpp/src/arrow/allocator.h       | 98 ++++++++++++++++++++++++++++++++++++
 cpp/src/arrow/memory_pool.h     |  6 +--
 4 files changed, 175 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/3b650014/cpp/src/arrow/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt
index ddeb81c..0abd4b9 100644
--- a/cpp/src/arrow/CMakeLists.txt
+++ b/cpp/src/arrow/CMakeLists.txt
@@ -17,6 +17,7 @@
 
 # Headers: top level
 install(FILES
+  allocator.h
   api.h
   array.h
   buffer.h
@@ -47,6 +48,7 @@ install(
 # Unit tests
 #######################################
 
+ADD_ARROW_TEST(allocator-test)
 ADD_ARROW_TEST(array-test)
 ADD_ARROW_TEST(array-decimal-test)
 ADD_ARROW_TEST(array-dictionary-test)

http://git-wip-us.apache.org/repos/asf/arrow/blob/3b650014/cpp/src/arrow/allocator-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/allocator-test.cc b/cpp/src/arrow/allocator-test.cc
new file mode 100644
index 0000000..0b24267
--- /dev/null
+++ b/cpp/src/arrow/allocator-test.cc
@@ -0,0 +1,72 @@
+// 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 "arrow/allocator.h"
+#include "arrow/test-util.h"
+
+namespace arrow {
+
+TEST(stl_allocator, MemoryTracking) {
+  auto pool = default_memory_pool();
+  stl_allocator<uint64_t> alloc;
+  uint64_t* data = alloc.allocate(100);
+
+  ASSERT_EQ(100 * sizeof(uint64_t), pool->bytes_allocated());
+
+  alloc.deallocate(data, 100);
+  ASSERT_EQ(0, pool->bytes_allocated());
+}
+
+#if !(defined(ARROW_VALGRIND) || defined(ADDRESS_SANITIZER))
+
+TEST(stl_allocator, TestOOM) {
+  stl_allocator<uint64_t> alloc;
+  uint64_t to_alloc = std::numeric_limits<uint64_t>::max();
+  ASSERT_THROW(alloc.allocate(to_alloc), std::bad_alloc);
+}
+
+TEST(stl_allocator, FreeLargeMemory) {
+  stl_allocator<uint8_t> alloc;
+
+  uint8_t* data = alloc.allocate(100);
+
+#ifndef NDEBUG
+  EXPECT_EXIT(alloc.deallocate(data, 120), ::testing::ExitedWithCode(1),
+      ".*Check failed: \\(bytes_allocated_\\) >= \\(size\\)");
+#endif
+
+  alloc.deallocate(data, 100);
+}
+
+TEST(stl_allocator, MaxMemory) {
+  DefaultMemoryPool pool;
+
+  ASSERT_EQ(0, pool.max_memory());
+  stl_allocator<uint8_t> alloc(&pool);
+  uint8_t* data = alloc.allocate(100);
+  uint8_t* data2 = alloc.allocate(100);
+
+  alloc.deallocate(data, 100);
+  alloc.deallocate(data2, 100);
+
+  ASSERT_EQ(200, pool.max_memory());
+}
+
+#endif  // ARROW_VALGRIND
+
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3b650014/cpp/src/arrow/allocator.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/allocator.h b/cpp/src/arrow/allocator.h
new file mode 100644
index 0000000..c976ba9
--- /dev/null
+++ b/cpp/src/arrow/allocator.h
@@ -0,0 +1,98 @@
+// 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_ALLOCATOR_H
+#define ARROW_ALLOCATOR_H
+
+#include <cstddef>
+#include <memory>
+#include <utility>
+#include "arrow/memory_pool.h"
+#include "arrow/status.h"
+
+namespace arrow {
+
+template <class T>
+class stl_allocator {
+ public:
+  using value_type = T;
+  using pointer = T*;
+  using const_pointer = const T*;
+  using reference = T&;
+  using const_reference = const T&;
+  using size_type = std::size_t;
+  using difference_type = std::ptrdiff_t;
+
+  template <class U>
+  struct rebind {
+    using other = stl_allocator<U>;
+  };
+
+  stl_allocator() noexcept : pool_(default_memory_pool()) {}
+  explicit stl_allocator(MemoryPool* pool) noexcept : pool_(pool) {}
+
+  template <class U>
+  stl_allocator(const stl_allocator<U>& rhs) noexcept : pool_(rhs.pool_) {}
+
+  ~stl_allocator() { pool_ = nullptr; }
+
+  pointer address(reference r) const noexcept { return std::addressof(r); }
+
+  const_pointer address(const_reference r) const noexcept { return std::addressof(r); }
+
+  pointer allocate(size_type n, const void* /*hint*/ = nullptr) {
+    uint8_t* data;
+    Status s = pool_->Allocate(n * sizeof(T), &data);
+    if (!s.ok()) throw std::bad_alloc();
+    return reinterpret_cast<pointer>(data);
+  }
+
+  void deallocate(pointer p, size_type n) {
+    pool_->Free(reinterpret_cast<uint8_t*>(p), n * sizeof(T));
+  }
+
+  size_type size_max() const noexcept { return size_type(-1) / sizeof(T); }
+
+  template <class U, class... Args>
+  void construct(U* p, Args&&... args) {
+    new (reinterpret_cast<void*>(p)) U(std::forward<Args>(args)...);
+  }
+
+  template <class U>
+  void destroy(U* p) {
+    p->~U();
+  }
+
+  MemoryPool* pool() const noexcept { return pool_; }
+
+ private:
+  MemoryPool* pool_;
+};
+
+template <class T1, class T2>
+bool operator==(const stl_allocator<T1>& lhs, const stl_allocator<T2>& rhs) noexcept {
+  return lhs.pool() == rhs.pool();
+}
+
+template <class T1, class T2>
+bool operator!=(const stl_allocator<T1>& lhs, const stl_allocator<T2>& rhs) noexcept {
+  return !(lhs == rhs);
+}
+
+}  // namespace arrow
+
+#endif  // ARROW_ALLOCATOR_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3b650014/cpp/src/arrow/memory_pool.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/memory_pool.h b/cpp/src/arrow/memory_pool.h
index 33d4c3e..0edfda6 100644
--- a/cpp/src/arrow/memory_pool.h
+++ b/cpp/src/arrow/memory_pool.h
@@ -15,8 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#ifndef ARROW_UTIL_MEMORY_POOL_H
-#define ARROW_UTIL_MEMORY_POOL_H
+#ifndef ARROW_MEMORY_POOL_H
+#define ARROW_MEMORY_POOL_H
 
 #include <atomic>
 #include <cstdint>
@@ -93,4 +93,4 @@ ARROW_EXPORT MemoryPool* default_memory_pool();
 
 }  // namespace arrow
 
-#endif  // ARROW_UTIL_MEMORY_POOL_H
+#endif  // ARROW_MEMORY_POOL_H