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/09/30 04:01:59 UTC

arrow git commit: ARROW-1600: [C++] Add Buffer constructor that wraps std::string

Repository: arrow
Updated Branches:
  refs/heads/master d4e09c765 -> 7c616114f


ARROW-1600: [C++] Add Buffer constructor that wraps std::string

Many other libraries interchange binary data with `std::string`. This makes it easy to wrap such data in an `arrow::Buffer`.

It may be worth adding a function that creates a buffer from a string, but owns its memory.

I also deprecated `arrow::GetBufferFromString`, which shouldn't have been public in the first place, since the new ctor is more general

Author: Wes McKinney <we...@twosigma.com>

Closes #1147 from wesm/ARROW-1600 and squashes the following commits:

f60f502c [Wes McKinney] Remove TestBuffer fixture
644bf2b7 [Wes McKinney] Add Buffer ctor that wraps std::string


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

Branch: refs/heads/master
Commit: 7c616114fb83d02faa5921db0296ca994a9b232b
Parents: d4e09c7
Author: Wes McKinney <we...@twosigma.com>
Authored: Fri Sep 29 23:01:42 2017 -0500
Committer: Wes McKinney <we...@twosigma.com>
Committed: Fri Sep 29 23:01:42 2017 -0500

----------------------------------------------------------------------
 cpp/src/arrow/buffer-test.cc             | 28 ++++++++++-------
 cpp/src/arrow/buffer.h                   | 44 +++++++++++++++++++--------
 cpp/src/arrow/ipc/ipc-json-test.cc       |  6 ++--
 cpp/src/arrow/ipc/ipc-read-write-test.cc |  8 ++---
 4 files changed, 55 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/7c616114/cpp/src/arrow/buffer-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/buffer-test.cc b/cpp/src/arrow/buffer-test.cc
index 334ad7b..5fd2706 100644
--- a/cpp/src/arrow/buffer-test.cc
+++ b/cpp/src/arrow/buffer-test.cc
@@ -32,9 +32,7 @@ using std::string;
 
 namespace arrow {
 
-class TestBuffer : public ::testing::Test {};
-
-TEST_F(TestBuffer, IsMutableFlag) {
+TEST(TestBuffer, IsMutableFlag) {
   Buffer buf(nullptr, 0);
 
   ASSERT_FALSE(buf.is_mutable());
@@ -46,7 +44,15 @@ TEST_F(TestBuffer, IsMutableFlag) {
   ASSERT_TRUE(pbuf.is_mutable());
 }
 
-TEST_F(TestBuffer, Resize) {
+TEST(TestBuffer, FromStdString) {
+  std::string val = "hello, world";
+
+  Buffer buf(val);
+  ASSERT_EQ(0, memcmp(buf.data(), val.c_str(), val.size()));
+  ASSERT_EQ(static_cast<int64_t>(val.size()), buf.size());
+}
+
+TEST(TestBuffer, Resize) {
   PoolBuffer buf;
 
   ASSERT_EQ(0, buf.size());
@@ -69,7 +75,7 @@ TEST_F(TestBuffer, Resize) {
   ASSERT_EQ(128, buf.capacity());
 }
 
-TEST_F(TestBuffer, TypedResize) {
+TEST(TestBuffer, TypedResize) {
   PoolBuffer buf;
 
   ASSERT_EQ(0, buf.size());
@@ -88,7 +94,7 @@ TEST_F(TestBuffer, TypedResize) {
   ASSERT_EQ(832, buf.capacity());
 }
 
-TEST_F(TestBuffer, ResizeOOM) {
+TEST(TestBuffer, ResizeOOM) {
 // This test doesn't play nice with AddressSanitizer
 #ifndef ADDRESS_SANITIZER
   // realloc fails, even though there may be no explicit limit
@@ -99,7 +105,7 @@ TEST_F(TestBuffer, ResizeOOM) {
 #endif
 }
 
-TEST_F(TestBuffer, EqualsWithSameContent) {
+TEST(TestBuffer, EqualsWithSameContent) {
   MemoryPool* pool = default_memory_pool();
   const int32_t bufferSize = 128 * 1024;
   uint8_t* rawBuffer1;
@@ -123,7 +129,7 @@ TEST_F(TestBuffer, EqualsWithSameContent) {
   pool->Free(rawBuffer3, bufferSize);
 }
 
-TEST_F(TestBuffer, EqualsWithSameBuffer) {
+TEST(TestBuffer, EqualsWithSameBuffer) {
   MemoryPool* pool = default_memory_pool();
   const int32_t bufferSize = 128 * 1024;
   uint8_t* rawBuffer;
@@ -142,7 +148,7 @@ TEST_F(TestBuffer, EqualsWithSameBuffer) {
   pool->Free(rawBuffer, bufferSize);
 }
 
-TEST_F(TestBuffer, Copy) {
+TEST(TestBuffer, Copy) {
   std::string data_str = "some data to copy";
 
   auto data = reinterpret_cast<const uint8_t*>(data_str.c_str());
@@ -157,7 +163,7 @@ TEST_F(TestBuffer, Copy) {
   ASSERT_TRUE(out->Equals(expected));
 }
 
-TEST_F(TestBuffer, SliceBuffer) {
+TEST(TestBuffer, SliceBuffer) {
   std::string data_str = "some data to slice";
 
   auto data = reinterpret_cast<const uint8_t*>(data_str.c_str());
@@ -171,7 +177,7 @@ TEST_F(TestBuffer, SliceBuffer) {
   ASSERT_EQ(2, buf.use_count());
 }
 
-TEST_F(TestBuffer, SliceMutableBuffer) {
+TEST(TestBuffer, SliceMutableBuffer) {
   std::string data_str = "some data to slice";
   auto data = reinterpret_cast<const uint8_t*>(data_str.c_str());
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/7c616114/cpp/src/arrow/buffer.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h
index d215267..dbd9376 100644
--- a/cpp/src/arrow/buffer.h
+++ b/cpp/src/arrow/buffer.h
@@ -47,9 +47,25 @@ class MemoryPool;
 /// The following invariant is always true: Size < Capacity
 class ARROW_EXPORT Buffer {
  public:
+  /// \brief Construct from buffer and size without copying memory
+  ///
+  /// \param[in] data a memory buffer
+  /// \param[in] size buffer size
+  ///
+  /// \note The passed memory must be kept alive through some other means
   Buffer(const uint8_t* data, int64_t size)
       : is_mutable_(false), data_(data), size_(size), capacity_(size) {}
 
+  /// \brief Construct from std::string without copying memory
+  ///
+  /// \param[in] data a std::string object
+  ///
+  /// \note The std::string must stay alive for the lifetime of the Buffer, so
+  /// temporary rvalue strings must be stored in an lvalue somewhere
+  explicit Buffer(const std::string& data)
+      : Buffer(reinterpret_cast<const uint8_t*>(data.c_str()),
+               static_cast<int64_t>(data.size())) {}
+
   virtual ~Buffer() = default;
 
   /// An offset into data that is owned by another buffer, but we want to be
@@ -69,6 +85,8 @@ class ARROW_EXPORT Buffer {
   /// Return true if both buffers are the same size and contain the same bytes
   /// up to the number of compared bytes
   bool Equals(const Buffer& other, int64_t nbytes) const;
+
+  /// Return true if both buffers are the same size and contain the same bytes
   bool Equals(const Buffer& other) const;
 
   /// Copy a section of the buffer into a new Buffer.
@@ -101,17 +119,6 @@ class ARROW_EXPORT Buffer {
   ARROW_DISALLOW_COPY_AND_ASSIGN(Buffer);
 };
 
-/// \brief Create Buffer referencing std::string memory
-///
-/// Warning: string instance must stay alive
-///
-/// \param str std::string instance
-/// \return std::shared_ptr<Buffer>
-static inline std::shared_ptr<Buffer> GetBufferFromString(const std::string& str) {
-  return std::make_shared<Buffer>(reinterpret_cast<const uint8_t*>(str.c_str()),
-                                  static_cast<int64_t>(str.size()));
-}
-
 /// Construct a view on passed buffer at the indicated offset and length. This
 /// function cannot fail and does not error checking (except in debug builds)
 static inline std::shared_ptr<Buffer> SliceBuffer(const std::shared_ptr<Buffer>& buffer,
@@ -331,11 +338,24 @@ Status AllocateResizableBuffer(MemoryPool* pool, const int64_t size,
                                std::shared_ptr<ResizableBuffer>* out);
 
 #ifndef ARROW_NO_DEPRECATED_API
+
 /// \deprecated Since 0.7.0
 ARROW_EXPORT
 Status AllocateBuffer(MemoryPool* pool, const int64_t size,
                       std::shared_ptr<MutableBuffer>* out);
-#endif
+
+/// \brief Create Buffer referencing std::string memory
+/// \deprecated Since 0.8.0
+///
+/// Warning: string instance must stay alive
+///
+/// \param str std::string instance
+/// \return std::shared_ptr<Buffer>
+static inline std::shared_ptr<Buffer> GetBufferFromString(const std::string& str) {
+  return std::make_shared<Buffer>(str);
+}
+
+#endif  // ARROW_NO_DEPRECATED_API
 
 }  // namespace arrow
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/7c616114/cpp/src/arrow/ipc/ipc-json-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/ipc-json-test.cc b/cpp/src/arrow/ipc/ipc-json-test.cc
index 7855aea..f2dd9e7 100644
--- a/cpp/src/arrow/ipc/ipc-json-test.cc
+++ b/cpp/src/arrow/ipc/ipc-json-test.cc
@@ -279,8 +279,7 @@ TEST(TestJsonFileReadWrite, BasicRoundTrip) {
 
   std::unique_ptr<JsonReader> reader;
 
-  auto buffer = std::make_shared<Buffer>(reinterpret_cast<const uint8_t*>(result.c_str()),
-                                         static_cast<int>(result.size()));
+  auto buffer = std::make_shared<Buffer>(result);
 
   ASSERT_OK(JsonReader::Open(buffer, &reader));
   ASSERT_TRUE(reader->schema()->Equals(*schema));
@@ -395,8 +394,7 @@ void CheckRoundtrip(const RecordBatch& batch) {
   std::string result;
   ASSERT_OK(writer->Finish(&result));
 
-  auto buffer = std::make_shared<Buffer>(reinterpret_cast<const uint8_t*>(result.c_str()),
-                                         static_cast<int64_t>(result.size()));
+  auto buffer = std::make_shared<Buffer>(result);
 
   std::unique_ptr<JsonReader> reader;
   ASSERT_OK(JsonReader::Open(buffer, &reader));

http://git-wip-us.apache.org/repos/asf/arrow/blob/7c616114/cpp/src/arrow/ipc/ipc-read-write-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/ipc-read-write-test.cc b/cpp/src/arrow/ipc/ipc-read-write-test.cc
index ad3af0f..d454d59 100644
--- a/cpp/src/arrow/ipc/ipc-read-write-test.cc
+++ b/cpp/src/arrow/ipc/ipc-read-write-test.cc
@@ -63,10 +63,10 @@ TEST(TestMessage, Equals) {
   std::string metadata = "foo";
   std::string body = "bar";
 
-  auto b1 = GetBufferFromString(metadata);
-  auto b2 = GetBufferFromString(metadata);
-  auto b3 = GetBufferFromString(body);
-  auto b4 = GetBufferFromString(body);
+  auto b1 = std::make_shared<Buffer>(metadata);
+  auto b2 = std::make_shared<Buffer>(metadata);
+  auto b3 = std::make_shared<Buffer>(body);
+  auto b4 = std::make_shared<Buffer>(body);
 
   Message msg1(b1, b3);
   Message msg2(b2, b4);