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 2019/06/22 20:22:55 UTC

[arrow] branch master updated: ARROW-4800: [C++] Introduce a Result class

This is an automated email from the ASF dual-hosted git repository.

wesm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 7e16ee4  ARROW-4800: [C++] Introduce a Result<T> class
7e16ee4 is described below

commit 7e16ee43ccccec6564078da78de746af3a18abdf
Author: Micah Kornfield <em...@gmail.com>
AuthorDate: Sat Jun 22 15:22:47 2019 -0500

    ARROW-4800: [C++] Introduce a Result<T> class
    
    - Mostly an adaptation of StatusOr from google/asylo (both header and unittests).
    - Demonstrate usage in ipc/writer*
    - If this PR is accepted I can do a follow-up PR to port over useful testing utilities.
    
    Author: Micah Kornfield <em...@gmail.com>
    Author: emkornfield <em...@gmail.com>
    
    Closes #4501 from emkornfield/error_or and squashes the following commits:
    
    82e48c453 <Micah Kornfield> fix linter.  Add unittest.
    aad79b183 <Micah Kornfield> rename to Return
    1d7dbfbcd <Micah Kornfield> Use bkietz's suggestion.  cleanup test
    d8e80431c <Micah Kornfield> fix compile errors
    cc626079c <Micah Kornfield> try non anonyous namespace
    86e43ac89 <Micah Kornfield> export before
    8a4b3ccf3 <Micah Kornfield> try explicit instantation for msvc
    f12f6d027 <Micah Kornfield> Revert "remove ARROW_EXPORT from test and try add link to gtest_main"
    9581b05b1 <Micah Kornfield> remove ARROW_EXPORT from test and try add link to gtest_main
    7a21e577a <Micah Kornfield> try exporting private test classes for appveyor
    0b44389da <Micah Kornfield> fix format
    de9d2d0d9 <Micah Kornfield> remove duplicate code.  fix format
    504fcd7bf <emkornfield> Update cpp/src/arrow/error_or.h
    31d9906c5 <Micah Kornfield> use vendored variant
    aa540da09 <Micah Kornfield> fix append
    6f459a5f9 <Micah Kornfield> address review comments
    7a1e54de4 <Micah Kornfield> Add Arrow export
    2886733fb <Micah Kornfield> use ARROW_RETURN_NOT_OK
    f7ed04f00 <Micah Kornfield> address comments
    3e2b3691a <Micah Kornfield> follow recommendation of docs for macro
    d5e43d034 <Micah Kornfield> ARROW-4800: Introduce an ErrorOr class
---
 LICENSE.txt                                     |  51 +++
 cpp/build-support/lint_cpp_cli.py               |   9 +-
 cpp/src/arrow/CMakeLists.txt                    |   2 +
 cpp/src/arrow/ipc/writer.cc                     |  33 +-
 cpp/src/arrow/ipc/writer.h                      |  27 ++
 cpp/src/arrow/public-api-test.cc                |   4 +
 cpp/src/arrow/result-test.cc                    | 533 ++++++++++++++++++++++++
 cpp/src/arrow/{public-api-test.cc => result.cc} |  22 +-
 cpp/src/arrow/result.h                          | 336 +++++++++++++++
 cpp/src/arrow/result_internal.h                 |  22 +
 10 files changed, 1019 insertions(+), 20 deletions(-)

diff --git a/LICENSE.txt b/LICENSE.txt
index 17e1d0b..3fe48de 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -882,3 +882,54 @@ This project include code from mingw-w64.
 Copyright (c) 2009 - 2013 by the mingw-w64 project
 Homepage: https://mingw-w64.org
 License: Zope Public License (ZPL) Version 2.1.
+
+---------------------------------------------------------------------------------
+
+This project include code from Google's Asylo project.
+
+* cpp/src/arrow/result.h is based on status_or.h
+
+Copyright (c)  Copyright 2017 Asylo authors
+Homepage: https://asylo.dev/ 
+License: Apache 2.0
+
+---------------------------------------------------------------------------------
+
+This project includes code from Google's protobuf project
+
+* cpp/src/arrow/result.h ARROW_ASSIGN_OR_RAISE is based off ASSIGN_OR_RETURN
+
+Copyright 2008 Google Inc.  All rights reserved.
+Homepage: https://developers.google.com/protocol-buffers/ 
+License: 
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it.  This code is not
+standalone and requires a support library to be linked with it.  This
+support library is itself covered by the above license.
diff --git a/cpp/build-support/lint_cpp_cli.py b/cpp/build-support/lint_cpp_cli.py
index aebe29e..f003e01 100644
--- a/cpp/build-support/lint_cpp_cli.py
+++ b/cpp/build-support/lint_cpp_cli.py
@@ -30,6 +30,7 @@ arguments = parser.parse_args()
 _STRIP_COMMENT_REGEX = re.compile('(.+)?(?=//)')
 _NULLPTR_REGEX = re.compile(r'.*\bnullptr\b.*')
 _RETURN_NOT_OK_REGEX = re.compile(r'.*\sRETURN_NOT_OK.*')
+_ASSIGN_OR_RAISE_REGEX = re.compile(r'.*\sASSIGN_OR_RAISE.*')
 
 
 def _paths(paths):
@@ -54,7 +55,13 @@ def lint_file(path):
          arrow/status.h
          test
          arrow/util/hash.h
-         arrow/python/util'''))
+         arrow/python/util''')),
+        (lambda x: re.match(_ASSIGN_OR_RAISE_REGEX, x),
+         'Use ARROW_ASSIGN_OR_RAISE in header files', _paths('''\
+         arrow/result_internal.h
+         test
+         '''))
+
     ]
 
     with open(path) as f:
diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt
index 1666d17..119ff2b 100644
--- a/cpp/src/arrow/CMakeLists.txt
+++ b/cpp/src/arrow/CMakeLists.txt
@@ -97,6 +97,7 @@ set(ARROW_SRCS
     memory_pool.cc
     pretty_print.cc
     record_batch.cc
+    result.cc
     scalar.cc
     sparse_tensor.cc
     status.cc
@@ -360,6 +361,7 @@ endif()
 add_arrow_test(memory_pool-test)
 add_arrow_test(pretty_print-test)
 add_arrow_test(public-api-test)
+add_arrow_test(result-test)
 add_arrow_test(scalar-test)
 add_arrow_test(status-test)
 add_arrow_test(stl-test)
diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc
index 37927a4..e1c2eca 100644
--- a/cpp/src/arrow/ipc/writer.cc
+++ b/cpp/src/arrow/ipc/writer.cc
@@ -36,6 +36,7 @@
 #include "arrow/ipc/util.h"
 #include "arrow/memory_pool.h"
 #include "arrow/record_batch.h"
+#include "arrow/result_internal.h"
 #include "arrow/sparse_tensor.h"
 #include "arrow/status.h"
 #include "arrow/table.h"
@@ -593,8 +594,8 @@ Status WriteRecordBatch(const RecordBatch& batch, int64_t buffer_start_offset,
 
 Status WriteRecordBatchStream(const std::vector<std::shared_ptr<RecordBatch>>& batches,
                               io::OutputStream* dst) {
-  std::shared_ptr<RecordBatchWriter> writer;
-  RETURN_NOT_OK(RecordBatchStreamWriter::Open(dst, batches[0]->schema(), &writer));
+  ASSIGN_OR_RAISE(std::shared_ptr<RecordBatchWriter> writer,
+                  RecordBatchStreamWriter::Open(dst, batches[0]->schema()));
   for (const auto& batch : batches) {
     // allow sizes > INT32_MAX
     DCHECK(batch->schema()->Equals(*batches[0]->schema())) << "Schemas unequal";
@@ -1159,11 +1160,16 @@ void RecordBatchStreamWriter::set_memory_pool(MemoryPool* pool) {
 Status RecordBatchStreamWriter::Open(io::OutputStream* sink,
                                      const std::shared_ptr<Schema>& schema,
                                      std::shared_ptr<RecordBatchWriter>* out) {
+  ASSIGN_OR_RAISE(*out, Open(sink, schema));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<RecordBatchWriter>> RecordBatchStreamWriter::Open(
+    io::OutputStream* sink, const std::shared_ptr<Schema>& schema) {
   // ctor is private
   auto result = std::shared_ptr<RecordBatchStreamWriter>(new RecordBatchStreamWriter());
   result->impl_.reset(new RecordBatchStreamWriterImpl(sink, schema));
-  *out = result;
-  return Status::OK();
+  return std::move(result);
 }
 
 Status RecordBatchStreamWriter::Close() { return impl_->Close(); }
@@ -1175,11 +1181,16 @@ RecordBatchFileWriter::~RecordBatchFileWriter() {}
 Status RecordBatchFileWriter::Open(io::OutputStream* sink,
                                    const std::shared_ptr<Schema>& schema,
                                    std::shared_ptr<RecordBatchWriter>* out) {
+  ASSIGN_OR_RAISE(*out, Open(sink, schema));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<RecordBatchWriter>> RecordBatchFileWriter::Open(
+    io::OutputStream* sink, const std::shared_ptr<Schema>& schema) {
   // ctor is private
   auto result = std::shared_ptr<RecordBatchFileWriter>(new RecordBatchFileWriter());
   result->file_impl_.reset(new RecordBatchFileWriterImpl(sink, schema));
-  *out = result;
-  return Status::OK();
+  return std::move(result);
 }
 
 Status RecordBatchFileWriter::WriteRecordBatch(const RecordBatch& batch,
@@ -1194,11 +1205,17 @@ namespace internal {
 Status OpenRecordBatchWriter(std::unique_ptr<IpcPayloadWriter> sink,
                              const std::shared_ptr<Schema>& schema,
                              std::unique_ptr<RecordBatchWriter>* out) {
-  out->reset(new RecordBatchPayloadWriter(std::move(sink), schema));
-  // XXX should we call Start()?
+  ASSIGN_OR_RAISE(*out, OpenRecordBatchWriter(std::move(sink), schema));
   return Status::OK();
 }
 
+Result<std::unique_ptr<RecordBatchWriter>> OpenRecordBatchWriter(
+    std::unique_ptr<IpcPayloadWriter> sink, const std::shared_ptr<Schema>& schema) {
+  // XXX should we call Start()?
+  return std::unique_ptr<RecordBatchWriter>(
+      new RecordBatchPayloadWriter(std::move(sink), schema));
+}
+
 }  // namespace internal
 
 // ----------------------------------------------------------------------
diff --git a/cpp/src/arrow/ipc/writer.h b/cpp/src/arrow/ipc/writer.h
index 6bb55db..d2b9faa 100644
--- a/cpp/src/arrow/ipc/writer.h
+++ b/cpp/src/arrow/ipc/writer.h
@@ -25,6 +25,7 @@
 #include <vector>
 
 #include "arrow/ipc/message.h"
+#include "arrow/result.h"
 #include "arrow/util/visibility.h"
 
 namespace arrow {
@@ -103,6 +104,15 @@ class ARROW_EXPORT RecordBatchStreamWriter : public RecordBatchWriter {
   static Status Open(io::OutputStream* sink, const std::shared_ptr<Schema>& schema,
                      std::shared_ptr<RecordBatchWriter>* out);
 
+  /// Create a new writer from stream sink and schema. User is responsible for
+  /// closing the actual OutputStream.
+  ///
+  /// \param[in] sink output stream to write to
+  /// \param[in] schema the schema of the record batches to be written
+  /// \return Result<std::shared_ptr<RecordBatchWriter>>
+  static Result<std::shared_ptr<RecordBatchWriter>> Open(
+      io::OutputStream* sink, const std::shared_ptr<Schema>& schema);
+
   /// \brief Write a record batch to the stream
   ///
   /// \param[in] batch the record batch to write
@@ -140,6 +150,14 @@ class ARROW_EXPORT RecordBatchFileWriter : public RecordBatchStreamWriter {
   static Status Open(io::OutputStream* sink, const std::shared_ptr<Schema>& schema,
                      std::shared_ptr<RecordBatchWriter>* out);
 
+  /// Create a new writer from stream sink and schema
+  ///
+  /// \param[in] sink output stream to write to
+  /// \param[in] schema the schema of the record batches to be written
+  /// \return Status
+  static Result<std::shared_ptr<RecordBatchWriter>> Open(
+      io::OutputStream* sink, const std::shared_ptr<Schema>& schema);
+
   /// \brief Write a record batch to the file
   ///
   /// \param[in] batch the record batch to write
@@ -332,6 +350,15 @@ Status OpenRecordBatchWriter(std::unique_ptr<IpcPayloadWriter> sink,
                              const std::shared_ptr<Schema>& schema,
                              std::unique_ptr<RecordBatchWriter>* out);
 
+/// Create a new RecordBatchWriter from IpcPayloadWriter and schema.
+///
+/// \param[in] sink the IpcPayloadWriter to write to
+/// \param[in] schema the schema of the record batches to be written
+/// \return Result<std::unique_ptr<RecordBatchWriter>>
+ARROW_EXPORT
+Result<std::unique_ptr<RecordBatchWriter>> OpenRecordBatchWriter(
+    std::unique_ptr<IpcPayloadWriter> sink, const std::shared_ptr<Schema>& schema);
+
 /// \brief Compute IpcPayload for the given schema
 /// \param[in] schema the Schema that is being serialized
 /// \param[in,out] dictionary_memo class to populate with assigned dictionary ids
diff --git a/cpp/src/arrow/public-api-test.cc b/cpp/src/arrow/public-api-test.cc
index 2009c2e..7ab7686 100644
--- a/cpp/src/arrow/public-api-test.cc
+++ b/cpp/src/arrow/public-api-test.cc
@@ -23,6 +23,10 @@
 #error "DCHECK should not be visible from Arrow public headers."
 #endif
 
+#ifdef ASSIGN_OR_RAISE
+#error "ASSIGN_OR_RAISE should not be visible from Arrow public headers."
+#endif
+
 #ifdef ARROW_UTIL_PARALLEL_H
 #error "arrow/util/parallel.h is an internal header"
 #endif
diff --git a/cpp/src/arrow/result-test.cc b/cpp/src/arrow/result-test.cc
new file mode 100644
index 0000000..36a7c8d
--- /dev/null
+++ b/cpp/src/arrow/result-test.cc
@@ -0,0 +1,533 @@
+// 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/result.h"
+
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+
+namespace {
+
+using ::testing::Eq;
+
+StatusCode kErrorCode = StatusCode::Invalid;
+constexpr char kErrorMessage[] = "Invalid argument";
+
+const int kIntElement = 42;
+constexpr char kStringElement[] =
+    "The Answer to the Ultimate Question of Life, the Universe, and Everything";
+
+// A data type without a default constructor.
+struct Foo {
+  int bar;
+  std::string baz;
+
+  explicit Foo(int value) : bar(value), baz(kStringElement) {}
+};
+
+// A data type with only copy constructors.
+struct CopyOnlyDataType {
+  explicit CopyOnlyDataType(int x) : data(x) {}
+
+  CopyOnlyDataType(const CopyOnlyDataType& other) = default;
+  CopyOnlyDataType& operator=(const CopyOnlyDataType& other) = default;
+
+  int data;
+};
+
+struct ImplicitlyCopyConvertible {
+  ImplicitlyCopyConvertible(const CopyOnlyDataType& co)  // NOLINT(runtime/explicit)
+      : copy_only(co) {}
+
+  CopyOnlyDataType copy_only;
+};
+
+// A data type with only move constructors.
+struct MoveOnlyDataType {
+  explicit MoveOnlyDataType(int x) : data(new int(x)) {}
+
+  MoveOnlyDataType(MoveOnlyDataType&& other) : data(other.data) { other.data = nullptr; }
+
+  MoveOnlyDataType(const MoveOnlyDataType& other) = delete;
+  MoveOnlyDataType& operator=(const MoveOnlyDataType& other) = delete;
+
+  ~MoveOnlyDataType() {
+    delete data;
+    data = nullptr;
+  }
+
+  int* data;
+};
+
+struct ImplicitlyMoveConvertible {
+  ImplicitlyMoveConvertible(MoveOnlyDataType&& mo)  // NOLINT(runtime/explicit)
+      : move_only(std::move(mo)) {}
+
+  MoveOnlyDataType move_only;
+};
+
+// A data type with dynamically-allocated data.
+struct HeapAllocatedObject {
+  int* value;
+
+  HeapAllocatedObject() {
+    value = new int;
+    *value = kIntElement;
+  }
+
+  HeapAllocatedObject(const HeapAllocatedObject& other) {
+    value = new int;
+    *value = *other.value;
+  }
+
+  HeapAllocatedObject& operator=(const HeapAllocatedObject& other) {
+    *value = *other.value;
+    return *this;
+  }
+
+  HeapAllocatedObject(HeapAllocatedObject&& other) {
+    value = other.value;
+    other.value = nullptr;
+  }
+
+  ~HeapAllocatedObject() { delete value; }
+};
+
+// Constructs a Foo.
+struct FooCtor {
+  using value_type = Foo;
+
+  Foo operator()() { return Foo(kIntElement); }
+};
+
+// Constructs a HeapAllocatedObject.
+struct HeapAllocatedObjectCtor {
+  using value_type = HeapAllocatedObject;
+
+  HeapAllocatedObject operator()() { return HeapAllocatedObject(); }
+};
+
+// Constructs an integer.
+struct IntCtor {
+  using value_type = int;
+
+  int operator()() { return kIntElement; }
+};
+
+// Constructs a string.
+struct StringCtor {
+  using value_type = std::string;
+
+  std::string operator()() { return std::string(kStringElement); }
+};
+
+// Constructs a vector of strings.
+struct StringVectorCtor {
+  using value_type = std::vector<std::string>;
+
+  std::vector<std::string> operator()() { return {kStringElement, kErrorMessage}; }
+};
+
+bool operator==(const Foo& lhs, const Foo& rhs) {
+  return (lhs.bar == rhs.bar) && (lhs.baz == rhs.baz);
+}
+
+bool operator==(const HeapAllocatedObject& lhs, const HeapAllocatedObject& rhs) {
+  return *lhs.value == *rhs.value;
+}
+
+// Returns an rvalue reference to the Result<T> object pointed to by
+// |result|.
+template <class T>
+Result<T>&& MoveResult(Result<T>* result) {
+  return std::move(*result);
+}
+
+// A test fixture is required for typed tests.
+template <typename T>
+class ResultTest : public ::testing::Test {};
+
+typedef ::testing::Types<IntCtor, FooCtor, StringCtor, StringVectorCtor,
+                         HeapAllocatedObjectCtor>
+    TestTypes;
+
+TYPED_TEST_CASE(ResultTest, TestTypes);
+
+// Verify that the default constructor for Result constructs an object with a
+// non-ok status.
+TYPED_TEST(ResultTest, ConstructorDefault) {
+  Result<typename TypeParam::value_type> result;
+  EXPECT_FALSE(result.ok());
+  EXPECT_EQ(result.status().code(), StatusCode::UnknownError);
+}
+
+// Verify that Result can be constructed from a Status object.
+TYPED_TEST(ResultTest, ConstructorStatus) {
+  Result<typename TypeParam::value_type> result(Status(kErrorCode, kErrorMessage));
+
+  EXPECT_FALSE(result.ok());
+  EXPECT_FALSE(result.status().ok());
+  EXPECT_EQ(result.status().code(), kErrorCode);
+  EXPECT_EQ(result.status().message(), kErrorMessage);
+}
+
+// Verify that Result can be constructed from an object of its element type.
+TYPED_TEST(ResultTest, ConstructorElementConstReference) {
+  typename TypeParam::value_type value = TypeParam()();
+  Result<typename TypeParam::value_type> result(value);
+
+  ASSERT_TRUE(result.ok());
+  ASSERT_TRUE(result.status().ok());
+  EXPECT_EQ(result.ValueOrDie(), value);
+  EXPECT_EQ(*result, value);
+}
+
+// Verify that Result can be constructed from an rvalue reference of an object
+// of its element type.
+TYPED_TEST(ResultTest, ConstructorElementRValue) {
+  typename TypeParam::value_type value = TypeParam()();
+  typename TypeParam::value_type value_copy(value);
+  Result<typename TypeParam::value_type> result(std::move(value));
+
+  ASSERT_TRUE(result.ok());
+  ASSERT_TRUE(result.status().ok());
+
+  // Compare to a copy of the original value, since the original was moved.
+  EXPECT_EQ(result.ValueOrDie(), value_copy);
+}
+
+// Verify that Result can be copy-constructed from a Result with a non-ok
+// status.
+TYPED_TEST(ResultTest, CopyConstructorNonOkStatus) {
+  Result<typename TypeParam::value_type> result1 = Status(kErrorCode, kErrorMessage);
+  Result<typename TypeParam::value_type> result2(result1);
+
+  EXPECT_EQ(result1.ok(), result2.ok());
+  EXPECT_EQ(result1.status().message(), result2.status().message());
+}
+
+// Verify that Result can be copy-constructed from a Result with an ok
+// status.
+TYPED_TEST(ResultTest, CopyConstructorOkStatus) {
+  Result<typename TypeParam::value_type> result1((TypeParam()()));
+  Result<typename TypeParam::value_type> result2(result1);
+
+  EXPECT_EQ(result1.ok(), result2.ok());
+  ASSERT_TRUE(result2.ok());
+  EXPECT_EQ(result1.ValueOrDie(), result2.ValueOrDie());
+  EXPECT_EQ(*result1, *result2);
+}
+
+// Verify that copy-assignment of a Result with a non-ok is working as
+// expected.
+TYPED_TEST(ResultTest, CopyAssignmentNonOkStatus) {
+  Result<typename TypeParam::value_type> result1(Status(kErrorCode, kErrorMessage));
+  Result<typename TypeParam::value_type> result2((TypeParam()()));
+
+  // Invoke the copy-assignment operator.
+  result2 = result1;
+  EXPECT_EQ(result1.ok(), result2.ok());
+  EXPECT_EQ(result1.status().message(), result2.status().message());
+}
+
+// Verify that copy-assignment of a Result with an ok status is working as
+// expected.
+TYPED_TEST(ResultTest, CopyAssignmentOkStatus) {
+  Result<typename TypeParam::value_type> result1((TypeParam()()));
+  Result<typename TypeParam::value_type> result2(Status(kErrorCode, kErrorMessage));
+
+  // Invoke the copy-assignment operator.
+  result2 = result1;
+  EXPECT_EQ(result1.ok(), result2.ok());
+  ASSERT_TRUE(result2.ok());
+  EXPECT_EQ(result1.ValueOrDie(), result2.ValueOrDie());
+  EXPECT_EQ(*result1, *result2);
+}
+
+// Verify that copy-assignment of a Result with a non-ok status to itself is
+// properly handled.
+TYPED_TEST(ResultTest, CopyAssignmentSelfNonOkStatus) {
+  Status status(kErrorCode, kErrorMessage);
+  Result<typename TypeParam::value_type> result(status);
+  result = *&result;
+
+  EXPECT_FALSE(result.ok());
+  EXPECT_EQ(result.status().code(), status.code());
+}
+
+// Verify that copy-assignment of a Result with an ok status to itself is
+// properly handled.
+TYPED_TEST(ResultTest, CopyAssignmentSelfOkStatus) {
+  typename TypeParam::value_type value = TypeParam()();
+  Result<typename TypeParam::value_type> result(value);
+  result = *&result;
+
+  ASSERT_TRUE(result.ok());
+  EXPECT_EQ(result.ValueOrDie(), value);
+  EXPECT_EQ(*result, value);
+}
+
+// Verify that Result can be move-constructed from a Result with a non-ok
+// status.
+TYPED_TEST(ResultTest, MoveConstructorNonOkStatus) {
+  Status status(kErrorCode, kErrorMessage);
+  Result<typename TypeParam::value_type> result1(status);
+  Result<typename TypeParam::value_type> result2(std::move(result1));
+
+  // Verify that the status of the donor object was updated.
+  EXPECT_FALSE(result1.ok());
+
+  // Verify that the destination object contains the status previously held by
+  // the donor.
+  EXPECT_FALSE(result2.ok());
+  EXPECT_EQ(result2.status().code(), status.code());
+}
+
+// Verify that Result can be move-constructed from a Result with an ok
+// status.
+TYPED_TEST(ResultTest, MoveConstructorOkStatus) {
+  typename TypeParam::value_type value = TypeParam()();
+  Result<typename TypeParam::value_type> result1(value);
+  Result<typename TypeParam::value_type> result2(std::move(result1));
+
+  // Verify that the donor object was updated to contain a non-ok status.
+  EXPECT_FALSE(result1.ok());
+
+  // The destination object should possess the value previously held by the
+  // donor.
+  ASSERT_TRUE(result2.ok());
+  EXPECT_EQ(result2.ValueOrDie(), value);
+}
+
+// Verify that move-assignment from a Result with a non-ok status is working
+// as expected.
+TYPED_TEST(ResultTest, MoveAssignmentOperatorNonOkStatus) {
+  Status status(kErrorCode, kErrorMessage);
+  Result<typename TypeParam::value_type> result1(status);
+  Result<typename TypeParam::value_type> result2((TypeParam()()));
+
+  // Invoke the move-assignment operator.
+  result2 = std::move(result1);
+
+  // Verify that the status of the donor object was updated.
+  EXPECT_FALSE(result1.ok());
+
+  // Verify that the destination object contains the status previously held by
+  // the donor.
+  EXPECT_FALSE(result2.ok());
+  EXPECT_EQ(result2.status().code(), status.code());
+}
+
+// Verify that move-assignment from a Result with an ok status is working as
+// expected.
+TYPED_TEST(ResultTest, MoveAssignmentOperatorOkStatus) {
+  typename TypeParam::value_type value = TypeParam()();
+  Result<typename TypeParam::value_type> result1(value);
+  Result<typename TypeParam::value_type> result2(Status(kErrorCode, kErrorMessage));
+
+  // Invoke the move-assignment operator.
+  result2 = std::move(result1);
+
+  // Verify that the donor object was updated to contain a non-ok status.
+  EXPECT_FALSE(result1.ok());
+
+  // The destination object should possess the value previously held by the
+  // donor.
+  ASSERT_TRUE(result2.ok());
+  EXPECT_EQ(result2.ValueOrDie(), value);
+}
+
+// Verify that move-assignment of a Result with a non-ok status to itself is
+// handled properly.
+TYPED_TEST(ResultTest, MoveAssignmentSelfNonOkStatus) {
+  Status status(kErrorCode, kErrorMessage);
+  Result<typename TypeParam::value_type> result(status);
+
+  result = MoveResult(&result);
+
+  EXPECT_FALSE(result.ok());
+  EXPECT_EQ(result.status().code(), status.code());
+}
+
+// Verify that move-assignment of a Result with an ok-status to itself is
+// handled properly.
+TYPED_TEST(ResultTest, MoveAssignmentSelfOkStatus) {
+  typename TypeParam::value_type value = TypeParam()();
+  Result<typename TypeParam::value_type> result(value);
+
+  result = MoveResult(&result);
+
+  ASSERT_TRUE(result.ok());
+  EXPECT_EQ(result.ValueOrDie(), value);
+}
+
+// Tests for move-only types. These tests use std::unique_ptr<> as the
+// test type, since it is valuable to support this type in the Asylo infra.
+// These tests are not part of the typed test suite for the following reasons:
+//   * std::unique_ptr<> cannot be used as a type in tests that expect
+//   the test type to support copy operations.
+//   * std::unique_ptr<> provides an equality operator that checks equality of
+//   the underlying ptr. Consequently, it is difficult to generalize existing
+//   tests that verify ValueOrDie() functionality using equality comparisons.
+
+// Verify that a Result object can be constructed from a move-only type.
+TEST(ResultTest, InitializationMoveOnlyType) {
+  std::string* str = new std::string(kStringElement);
+  std::unique_ptr<std::string> value(str);
+  Result<std::unique_ptr<std::string>> result(std::move(value));
+
+  ASSERT_TRUE(result.ok());
+  EXPECT_EQ(result.ValueOrDie().get(), str);
+}
+
+// Verify that a Result object can be move-constructed from a move-only type.
+TEST(ResultTest, MoveConstructorMoveOnlyType) {
+  std::string* str = new std::string(kStringElement);
+  std::unique_ptr<std::string> value(str);
+  Result<std::unique_ptr<std::string>> result1(std::move(value));
+  Result<std::unique_ptr<std::string>> result2(std::move(result1));
+
+  // Verify that the donor object was updated to contain a non-ok status.
+  EXPECT_FALSE(result1.ok());
+
+  // The destination object should possess the value previously held by the
+  // donor.
+  ASSERT_TRUE(result2.ok());
+  EXPECT_EQ(result2.ValueOrDie().get(), str);
+}
+
+// Verify that a Result object can be move-assigned to from a Result object
+// containing a move-only type.
+TEST(ResultTest, MoveAssignmentMoveOnlyType) {
+  std::string* str = new std::string(kStringElement);
+  std::unique_ptr<std::string> value(str);
+  Result<std::unique_ptr<std::string>> result1(std::move(value));
+  Result<std::unique_ptr<std::string>> result2(Status(kErrorCode, kErrorMessage));
+
+  // Invoke the move-assignment operator.
+  result2 = std::move(result1);
+
+  // Verify that the donor object was updated to contain a non-ok status.
+  EXPECT_FALSE(result1.ok());
+
+  // The destination object should possess the value previously held by the
+  // donor.
+  ASSERT_TRUE(result2.ok());
+  EXPECT_EQ(result2.ValueOrDie().get(), str);
+}
+
+// Verify that a value can be moved out of a Result object via ValueOrDie().
+TEST(ResultTest, ValueOrDieMovedValue) {
+  std::string* str = new std::string(kStringElement);
+  std::unique_ptr<std::string> value(str);
+  Result<std::unique_ptr<std::string>> result(std::move(value));
+
+  std::unique_ptr<std::string> moved_value = std::move(result).ValueOrDie();
+  EXPECT_EQ(moved_value.get(), str);
+  EXPECT_EQ(*moved_value, kStringElement);
+
+  // Verify that the Result object was invalidated after the value was moved.
+  EXPECT_FALSE(result.ok());
+}
+
+// Verify that a Result<T> is implicitly constructible from some U, where T is
+// a type which has an implicit constructor taking a const U &.
+TEST(ResultTest, TemplateValueCopyConstruction) {
+  CopyOnlyDataType copy_only(kIntElement);
+  Result<ImplicitlyCopyConvertible> result(copy_only);
+
+  EXPECT_TRUE(result.ok());
+  EXPECT_EQ(result.ValueOrDie().copy_only.data, kIntElement);
+}
+
+// Verify that a Result<T> is implicitly constructible from some U, where T is
+// a type which has an implicit constructor taking a U &&.
+TEST(ResultTest, TemplateValueMoveConstruction) {
+  MoveOnlyDataType move_only(kIntElement);
+  Result<ImplicitlyMoveConvertible> result(std::move(move_only));
+
+  EXPECT_TRUE(result.ok());
+  EXPECT_EQ(*result.ValueOrDie().move_only.data, kIntElement);
+}
+
+// Verify that a Result<U> is assignable to a Result<T>, where T
+// is a type which has an implicit constructor taking a const U &.
+TEST(ResultTest, TemplateCopyAssign) {
+  CopyOnlyDataType copy_only(kIntElement);
+  Result<CopyOnlyDataType> result(copy_only);
+
+  Result<ImplicitlyCopyConvertible> result2 = result;
+
+  EXPECT_TRUE(result.ok());
+  EXPECT_EQ(result.ValueOrDie().data, kIntElement);
+  EXPECT_TRUE(result2.ok());
+  EXPECT_EQ(result2.ValueOrDie().copy_only.data, kIntElement);
+}
+
+// Verify that a Result<U> is assignable to a Result<T>, where T is a type
+// which has an implicit constructor taking a U &&.
+TEST(ResultTest, TemplateMoveAssign) {
+  MoveOnlyDataType move_only(kIntElement);
+  Result<MoveOnlyDataType> result(std::move(move_only));
+
+  Result<ImplicitlyMoveConvertible> result2 = std::move(result);
+
+  EXPECT_TRUE(result2.ok());
+  EXPECT_EQ(*result2.ValueOrDie().move_only.data, kIntElement);
+
+  //  NOLINTNEXTLINE use after move.
+  EXPECT_FALSE(result.ok());
+  //  NOLINTNEXTLINE use after move.
+}
+
+// Verify that a Result<U> is constructible from a Result<T>, where T is a
+// type which has an implicit constructor taking a const U &.
+TEST(ResultTest, TemplateCopyConstruct) {
+  CopyOnlyDataType copy_only(kIntElement);
+  Result<CopyOnlyDataType> result(copy_only);
+  Result<ImplicitlyCopyConvertible> result2(result);
+
+  EXPECT_TRUE(result.ok());
+  EXPECT_EQ(result.ValueOrDie().data, kIntElement);
+  EXPECT_TRUE(result2.ok());
+  EXPECT_EQ(result2.ValueOrDie().copy_only.data, kIntElement);
+}
+
+// Verify that a Result<U> is constructible from a Result<T>, where T is a
+// type which has an implicit constructor taking a U &&.
+TEST(ResultTest, TemplateMoveConstruct) {
+  MoveOnlyDataType move_only(kIntElement);
+  Result<MoveOnlyDataType> result(std::move(move_only));
+  Result<ImplicitlyMoveConvertible> result2(std::move(result));
+
+  EXPECT_TRUE(result2.ok());
+  EXPECT_EQ(*result2.ValueOrDie().move_only.data, kIntElement);
+
+  //  NOLINTNEXTLINE use after move.
+  EXPECT_FALSE(result.ok());
+  //  NOLINTNEXTLINE use after move.
+}
+
+}  // namespace
+}  // namespace arrow
diff --git a/cpp/src/arrow/public-api-test.cc b/cpp/src/arrow/result.cc
similarity index 69%
copy from cpp/src/arrow/public-api-test.cc
copy to cpp/src/arrow/result.cc
index 2009c2e..9d2926d 100644
--- a/cpp/src/arrow/public-api-test.cc
+++ b/cpp/src/arrow/result.cc
@@ -15,18 +15,18 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "arrow/api.h"      // IWYU pragma: keep
-#include "arrow/io/api.h"   // IWYU pragma: keep
-#include "arrow/ipc/api.h"  // IWYU pragma: keep
+#include "arrow/result.h"
 
-#ifdef DCHECK
-#error "DCHECK should not be visible from Arrow public headers."
-#endif
+#include <string>
 
-#ifdef ARROW_UTIL_PARALLEL_H
-#error "arrow/util/parallel.h is an internal header"
-#endif
+#include "arrow/util/logging.h"
 
-#include <gtest/gtest.h>
+namespace arrow {
 
-TEST(_, _) {}
+namespace internal {
+
+void DieWithMessage(const std::string& msg) { ARROW_LOG(FATAL) << msg; }
+
+}  // namespace internal
+
+}  // namespace arrow
diff --git a/cpp/src/arrow/result.h b/cpp/src/arrow/result.h
new file mode 100644
index 0000000..f01513a
--- /dev/null
+++ b/cpp/src/arrow/result.h
@@ -0,0 +1,336 @@
+//
+// Copyright 2017 Asylo authors
+//
+// Licensed 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.
+//
+
+// Adapted from Asylo
+
+#pragma once
+
+#include <string>
+#include <utility>
+
+#include "arrow/status.h"
+#include "arrow/util/macros.h"
+#include "arrow/util/variant.h"
+
+namespace arrow {
+
+namespace internal {
+
+ARROW_EXPORT void DieWithMessage(const std::string& msg);
+
+}  // namespace internal
+
+// A class for representing either a usable value, or an error.
+///
+/// A Result object either contains a value of type `T` or a Status object
+/// explaining why such a value is not present. The type `T` must be
+/// copy-constructible and/or move-constructible.
+///
+/// The state of a Result object may be determined by calling ok() or
+/// status(). The ok() method returns true if the object contains a valid value.
+/// The status() method returns the internal Status object. A Result object
+/// that contains a valid value will return an OK Status for a call to status().
+///
+/// A value of type `T` may be extracted from a Result object through a call
+/// to ValueOrDie(). This function should only be called if a call to ok()
+/// returns true. Sample usage:
+///
+/// ```
+///   arrow::Result<Foo> result = CalculateFoo();
+///   if (result.ok()) {
+///     Foo foo = result.ValueOrDie();
+///     foo.DoSomethingCool();
+///   } else {
+///     ARROW_LOG(ERROR) << result.status();
+///  }
+/// ```
+///
+/// If `T` is a move-only type, like `std::unique_ptr<>`, then the value should
+/// only be extracted after invoking `std::move()` on the Result object.
+/// Sample usage:
+///
+/// ```
+///   arrow::Result<std::unique_ptr<Foo>> result = CalculateFoo();
+///   if (result.ok()) {
+///     std::unique_ptr<Foo> foo = std::move(result).ValueOrDie();
+///     foo->DoSomethingCool();
+///   } else {
+///     ARROW_LOG(ERROR) << result.status();
+///   }
+/// ```
+///
+/// Result is provided for the convenience of implementing functions that
+/// return some value but may fail during execution. For instance, consider a
+/// function with the following signature:
+///
+/// ```
+///   arrow::Status CalculateFoo(int *output);
+/// ```
+///
+/// This function may instead be written as:
+///
+/// ```
+///   arrow::Result<int> CalculateFoo();
+/// ```
+template <class T>
+class Result {
+  template <typename U>
+  friend class Result;
+  using VariantType = arrow::util::variant<T, Status, const char*>;
+  static_assert(!std::is_same<T, Status>::value,
+                "this assert indicates you have probably made a metaprogramming error");
+
+ public:
+  /// Constructs a Result object that contains a non-OK status.
+  ///
+  /// This constructor is marked `explicit` to prevent attempts to `return {}`
+  /// from a function with a return type of, for example,
+  /// `Result<std::vector<int>>`. While `return {}` seems like it would return
+  /// an empty vector, it will actually invoke the default constructor of
+  /// Result.
+  explicit Result()  // NOLINT(runtime/explicit)
+      : variant_(Status::UnknownError("Unknown error")) {}
+
+  ~Result() = default;
+
+  /// Constructs a Result object with the given non-OK Status object. All
+  /// calls to ValueOrDie() on this object will abort. The given `status` must
+  /// not be an OK status, otherwise this constructor will abort.
+  ///
+  /// This constructor is not declared explicit so that a function with a return
+  /// type of `Result<T>` can return a Status object, and the status will be
+  /// implicitly converted to the appropriate return type as a matter of
+  /// convenience.
+  ///
+  /// \param status The non-OK Status object to initalize to.
+  Result(const Status& status)  // NOLINT(runtime/explicit)
+      : variant_(status) {
+    if (ARROW_PREDICT_FALSE(status.ok())) {
+      internal::DieWithMessage(std::string("Constructed with a non-error status: ") +
+                               status.ToString());
+    }
+  }
+
+  /// Constructs a Result object that contains `value`. The resulting object
+  /// is considered to have an OK status. The wrapped element can be accessed
+  /// with ValueOrDie().
+  ///
+  /// This constructor is made implicit so that a function with a return type of
+  /// `Result<T>` can return an object of type `U &&`, implicitly converting
+  /// it to a `Result<T>` object.
+  ///
+  /// Note that `T` must be implicitly constructible from `U`, and `U` must not
+  /// be a (cv-qualified) Status or Status-reference type. Due to C++
+  /// reference-collapsing rules and perfect-forwarding semantics, this
+  /// constructor matches invocations that pass `value` either as a const
+  /// reference or as an rvalue reference. Since Result needs to work for both
+  /// reference and rvalue-reference types, the constructor uses perfect
+  /// forwarding to avoid invalidating arguments that were passed by reference.
+  /// See http://thbecker.net/articles/rvalue_references/section_08.html for
+  /// additional details.
+  ///
+  /// \param value The value to initialize to.
+  template <typename U,
+            typename E = typename std::enable_if<
+                std::is_constructible<T, U>::value && std::is_convertible<U, T>::value &&
+                !std::is_same<typename std::remove_reference<
+                                  typename std::remove_cv<U>::type>::type,
+                              Status>::value>::type>
+  Result(U&& value)  // NOLINT(runtime/explicit)
+      : variant_(std::forward<U>(value)) {}
+
+  /// Copy constructor.
+  ///
+  /// This constructor needs to be explicitly defined because the presence of
+  /// the move-assignment operator deletes the default copy constructor. In such
+  /// a scenario, since the deleted copy constructor has stricter binding rules
+  /// than the templated copy constructor, the templated constructor cannot act
+  /// as a copy constructor, and any attempt to copy-construct a `Result`
+  /// object results in a compilation error.
+  ///
+  /// \param other The value to copy from.
+  Result(const Result& other) = default;
+
+  /// Templatized constructor that constructs a `Result<T>` from a const
+  /// reference to a `Result<U>`.
+  ///
+  /// `T` must be implicitly constructible from `const U &`.
+  ///
+  /// \param other The value to copy from.
+  template <typename U,
+            typename E = typename std::enable_if<std::is_constructible<T, U>::value &&
+                                                 std::is_convertible<U, T>::value>::type>
+  Result(const Result<U>& other) : variant_("unitialized") {
+    AssignVariant(other.variant_);
+  }
+
+  /// Copy-assignment operator.
+  ///
+  /// \param other The Result object to copy.
+  Result& operator=(const Result& other) = default;
+
+  /// Templatized constructor which constructs a `Result<T>` by moving the
+  /// contents of a `Result<U>`. `T` must be implicitly constructible from `U
+  /// &&`.
+  ///
+  /// Sets `other` to contain a non-OK status with a`StatusError::Invalid`
+  /// error code.
+  ///
+  /// \param other The Result object to move from and set to a non-OK status.
+  template <typename U,
+            typename E = typename std::enable_if<std::is_constructible<T, U>::value &&
+                                                 std::is_convertible<U, T>::value>::type>
+  Result(Result<U>&& other) : variant_("unitialized") {
+    AssignVariant(std::move(other.variant_));
+    other.variant_ = "Value was moved to another Result.";
+  }
+
+  /// Move-assignment operator.
+  ///
+  /// Sets `other` to an invalid state..
+  ///
+  /// \param other The Result object to assign from and set to a non-OK
+  /// status.
+  Result& operator=(Result&& other) {
+    // Check for self-assignment.
+    if (this == &other) {
+      return *this;
+    }
+    AssignVariant(std::move(other.variant_));
+    other.variant_ = "Value was moved to another Result.";
+
+    return *this;
+  }
+
+  /// Indicates whether the object contains a `T` value.  Generally instead
+  /// of accessing this directly you will want to use ASSIGN_OR_RAISE defined
+  /// below.
+  ///
+  /// \return True if this Result object's status is OK (i.e. a call to ok()
+  /// returns true). If this function returns true, then it is safe to access
+  /// the wrapped element through a call to ValueOrDie().
+  bool ok() const { return arrow::util::holds_alternative<T>(variant_); }
+
+  /// \brief Equivelant to ok().
+  // operator bool() const { return ok(); }
+
+  /// Gets the stored status object, or an OK status if a `T` value is stored.
+  ///
+  /// \return The stored non-OK status object, or an OK status if this object
+  ///         has a value.
+  Status status() const {
+    return ok() ? Status::OK() : arrow::util::get<Status>(variant_);
+  }
+
+  /// Gets the stored `T` value.
+  ///
+  /// This method should only be called if this Result object's status is OK
+  /// (i.e. a call to ok() returns true), otherwise this call will abort.
+  ///
+  /// \return The stored `T` value.
+  const T& ValueOrDie() const& {
+    if (ARROW_PREDICT_FALSE(!ok())) {
+      internal::DieWithMessage(std::string("ValueOrDie called on an error: ") +
+                               status().ToString());
+    }
+    return arrow::util::get<T>(variant_);
+  }
+  const T& operator*() const& { return ValueOrDie(); }
+
+  /// Gets a mutable reference to the stored `T` value.
+  ///
+  /// This method should only be called if this Result object's status is OK
+  /// (i.e. a call to ok() returns true), otherwise this call will abort.
+  ///
+  /// \return The stored `T` value.
+  T& ValueOrDie() & {
+    if (ARROW_PREDICT_FALSE(!ok())) {
+      internal::DieWithMessage(std::string("ValueOrDie called on an error: ") +
+                               status().ToString());
+    }
+    return arrow::util::get<T>(variant_);
+  }
+  T& operator*() & { return ValueOrDie(); }
+
+  /// Moves and returns the internally-stored `T` value.
+  ///
+  /// This method should only be called if this Result object's status is OK
+  /// (i.e. a call to ok() returns true), otherwise this call will abort. The
+  /// Result object is invalidated after this call and will be updated to
+  /// contain a non-OK status.
+  ///
+  /// \return The stored `T` value.
+  T ValueOrDie() && {
+    if (ARROW_PREDICT_FALSE(!ok())) {
+      internal::DieWithMessage(std::string("ValueOrDie called on an error: ") +
+                               status().ToString());
+    }
+    T tmp(std::move(arrow::util::get<T>(variant_)));
+    variant_ = "Object already returned with ValueOrDie";
+    return std::move(tmp);
+  }
+  T operator*() && { return std::move(ValueOrDie()); }
+
+ private:
+  // Assignment is disabled by default so we need to destruct/reconstruct
+  // the value.
+  template <typename U>
+  void AssignVariant(arrow::util::variant<U, Status, const char*>&& other) {
+    variant_.~variant();
+    if (arrow::util::holds_alternative<U>(other)) {
+      // Reuse memory of variant_ for construction
+      new (&variant_) VariantType(arrow::util::get<U>(std::move(other)));
+    } else {
+      new (&variant_) VariantType(arrow::util::get<Status>(std::move(other)));
+    }
+  }
+
+  // Assignment is disabled by default so we need to destruct/reconstruct
+  // the value.
+  template <typename U>
+  void AssignVariant(const arrow::util::variant<U, Status, const char*>& other) {
+    variant_.~variant();
+    if (arrow::util::holds_alternative<U>(other)) {
+      // Reuse memory of variant_ for construction
+      new (&variant_) VariantType(arrow::util::get<U>(other));
+    } else {
+      new (&variant_) VariantType(arrow::util::get<Status>(other));
+    }
+  }
+
+  arrow::util::variant<T, Status, const char*> variant_;
+};
+
+#define ARROW_ASSIGN_OR_RAISE_IMPL(status_name, lhs, rexpr) \
+  auto status_name = (rexpr);                               \
+  ARROW_RETURN_NOT_OK(status_name.status());                \
+  lhs = std::move(status_name).ValueOrDie();
+
+#define ARROW_ASSIGN_OR_RAISE_NAME(x, y) ARROW_CONCAT(x, y)
+
+// Executes an expression that returns a Result, extracting its value
+// into the variable defined by lhs (or returning on error).
+//
+// Example: Assigning to an existing value
+//   ValueType value;
+//   ARROW_ASSIGN_OR_RAISE(value, MaybeGetValue(arg));
+//
+// WARNING: ASSIGN_OR_RAISE expands into multiple statements; it cannot be used
+//  in a single statement (e.g. as the body of an if statement without {})!
+#define ARROW_ASSIGN_OR_RAISE(lhs, rexpr)                                              \
+  ARROW_ASSIGN_OR_RAISE_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
+                             lhs, rexpr);
+}  // namespace arrow
diff --git a/cpp/src/arrow/result_internal.h b/cpp/src/arrow/result_internal.h
new file mode 100644
index 0000000..7550f94
--- /dev/null
+++ b/cpp/src/arrow/result_internal.h
@@ -0,0 +1,22 @@
+//
+// Copyright 2017 Asylo authors
+//
+// Licensed 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.
+//
+#pragma once
+
+#include "arrow/result.h"
+
+#ifndef ASSIGN_OR_RAISE
+#define ASSIGN_OR_RAISE(lhs, rhs) ARROW_ASSIGN_OR_RAISE(lhs, rhs)
+#endif