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/03 22:28:57 UTC

arrow git commit: ARROW-1449: Implement Decimal using only Int128

Repository: arrow
Updated Branches:
  refs/heads/master 08b41f934 -> 6f2fd8dbf


ARROW-1449: Implement Decimal using only Int128

Author: Phillip Cloud <cp...@gmail.com>

Closes #1028 from cpcloud/decimal-single-representation and squashes the following commits:

2ee85a1e [Phillip Cloud] Formatting
0dfceff5 [Phillip Cloud] Don't assign a new string if not necessary
25853e94 [Phillip Cloud] Use STL instead of while loop
bd5e6838 [Phillip Cloud] Fix appveyor warning from stati_cast
04368e56 [Phillip Cloud] Move template implementation to header
18c59489 [Phillip Cloud] Static cast result of std::isdigit to bool
6e967757 [Phillip Cloud] Fix macros
f72f5466 [Phillip Cloud] Formatting
759247e4 [Phillip Cloud] Remove unused header
41fc9a0f [Phillip Cloud] IWYU
ccac0bda [Phillip Cloud] Fix complaint about bool -> Int128
9908d9c3 [Phillip Cloud] Formatting
3d8e399e [Phillip Cloud] Ignore short and long usage linting in decimal-test.cc
8cfa1785 [Phillip Cloud] Make sure we can construct from any integral type
cd7189fa [Phillip Cloud] ARROW-1449: Implement Decimal using only Int128


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

Branch: refs/heads/master
Commit: 6f2fd8dbf727a6155406e507ee378c86698f976c
Parents: 08b41f9
Author: Phillip Cloud <cp...@gmail.com>
Authored: Sun Sep 3 18:28:50 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Sun Sep 3 18:28:50 2017 -0400

----------------------------------------------------------------------
 cpp/src/arrow/array-decimal-test.cc     | 190 ++++++++-------------------
 cpp/src/arrow/array.cc                  |  23 +---
 cpp/src/arrow/buffer.h                  |  14 ++
 cpp/src/arrow/builder.cc                |  23 +---
 cpp/src/arrow/builder.h                 |  14 +-
 cpp/src/arrow/compare.cc                |  67 ++--------
 cpp/src/arrow/python/arrow_to_pandas.cc |  49 +------
 cpp/src/arrow/python/builtin_convert.cc |  37 ++----
 cpp/src/arrow/python/helpers.cc         |   4 +-
 cpp/src/arrow/python/pandas_to_arrow.cc |  27 +---
 cpp/src/arrow/type-test.cc              |  14 +-
 cpp/src/arrow/type.h                    |  15 +--
 cpp/src/arrow/util/decimal-test.cc      | 190 ++++++++++++++-------------
 cpp/src/arrow/util/decimal.cc           | 149 +++++++++------------
 cpp/src/arrow/util/decimal.h            | 110 +---------------
 cpp/src/arrow/util/int128.cc            |  45 ++++---
 cpp/src/arrow/util/int128.h             |  55 ++++----
 cpp/src/arrow/util/macros.h             |   4 +-
 18 files changed, 340 insertions(+), 690 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/array-decimal-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array-decimal-test.cc b/cpp/src/arrow/array-decimal-test.cc
index 7bbf2f0..5f3a4f1 100644
--- a/cpp/src/arrow/array-decimal-test.cc
+++ b/cpp/src/arrow/array-decimal-test.cc
@@ -24,12 +24,9 @@
 
 #include "arrow/array.h"
 #include "arrow/builder.h"
-#include "arrow/memory_pool.h"
-#include "arrow/status.h"
 #include "arrow/test-util.h"
-#include "arrow/type.h"
-#include "arrow/util/bit-util.h"
 #include "arrow/util/decimal.h"
+#include "arrow/util/int128.h"
 
 using std::size_t;
 
@@ -39,180 +36,97 @@ class Buffer;
 
 namespace decimal {
 
-template <typename T>
-class DecimalTestBase {
- public:
-  DecimalTestBase() : pool_(default_memory_pool()) {}
+using DecimalVector = std::vector<Int128>;
 
-  virtual std::vector<uint8_t> MakeData(const std::vector<T>& input,
-                                        size_t byte_width) const = 0;
+class DecimalTest : public ::testing::TestWithParam<int> {
+ public:
+  DecimalTest() {}
 
-  void InitBuilder(const std::shared_ptr<DecimalType>& type, const std::vector<T>& draw,
-                   const std::vector<uint8_t>& valid_bytes, int byte_width,
-                   std::shared_ptr<DecimalBuilder>* builder, size_t* null_count) const {
-    *builder = std::make_shared<DecimalBuilder>(type, pool_);
+  template <size_t BYTE_WIDTH = 16>
+  void MakeData(const DecimalVector& input, std::vector<uint8_t>* out) const {
+    out->reserve(input.size() * BYTE_WIDTH);
 
-    size_t size = draw.size();
-    ASSERT_OK((*builder)->Reserve(size));
+    std::array<uint8_t, BYTE_WIDTH> bytes{{0}};
 
-    for (size_t i = 0; i < size; ++i) {
-      if (valid_bytes[i]) {
-        ASSERT_OK((*builder)->Append(draw[i]));
-      } else {
-        ASSERT_OK((*builder)->AppendNull());
-        ++*null_count;
-      }
+    for (const auto& value : input) {
+      ASSERT_OK(value.ToBytes(&bytes));
+      out->insert(out->end(), bytes.cbegin(), bytes.cend());
     }
   }
 
-  void TestCreate(int precision, const std::vector<T>& draw,
+  template <size_t BYTE_WIDTH = 16>
+  void TestCreate(int precision, const DecimalVector& draw,
                   const std::vector<uint8_t>& valid_bytes, int64_t offset) const {
     auto type = std::make_shared<DecimalType>(precision, 4);
 
-    std::shared_ptr<DecimalBuilder> builder;
+    auto builder = std::make_shared<DecimalBuilder>(type);
 
     size_t null_count = 0;
 
     const size_t size = draw.size();
-    const int byte_width = type->byte_width();
-
-    InitBuilder(type, draw, valid_bytes, byte_width, &builder, &null_count);
 
-    auto raw_bytes = MakeData(draw, static_cast<size_t>(byte_width));
-    auto expected_data = std::make_shared<Buffer>(raw_bytes.data(), size * byte_width);
-    std::shared_ptr<Buffer> expected_null_bitmap;
-    ASSERT_OK(BitUtil::BytesToBits(valid_bytes, &expected_null_bitmap));
+    ASSERT_OK(builder->Reserve(size));
 
-    int64_t expected_null_count = test::null_count(valid_bytes);
-    auto expected = std::make_shared<DecimalArray>(
-        type, size, expected_data, expected_null_bitmap, expected_null_count, 0);
+    for (size_t i = 0; i < size; ++i) {
+      if (valid_bytes[i]) {
+        ASSERT_OK(builder->Append(draw[i]));
+      } else {
+        ASSERT_OK(builder->AppendNull());
+        ++null_count;
+      }
+    }
 
     std::shared_ptr<Array> out;
     ASSERT_OK(builder->Finish(&out));
-    ASSERT_TRUE(out->Slice(offset)->Equals(
-        *expected->Slice(offset, expected->length() - offset)));
-  }
-
- private:
-  MemoryPool* pool_;
-};
-
-template <typename T>
-class DecimalTest : public DecimalTestBase<T> {
- public:
-  std::vector<uint8_t> MakeData(const std::vector<T>& input,
-                                size_t byte_width) const override {
-    std::vector<uint8_t> result(input.size() * byte_width);
-    // TODO(phillipc): There's probably a better way to do this
-    constexpr static const size_t bytes_per_element = sizeof(T);
-    for (size_t i = 0, j = 0; i < input.size(); ++i, j += bytes_per_element) {
-      *reinterpret_cast<typename T::value_type*>(&result[j]) = input[i].value;
-    }
-    return result;
-  }
-};
-
-template <>
-class DecimalTest<Decimal128> : public DecimalTestBase<Decimal128> {
- public:
-  std::vector<uint8_t> MakeData(const std::vector<Decimal128>& input,
-                                size_t byte_width) const override {
-    std::vector<uint8_t> result;
-    result.reserve(input.size() * byte_width);
-    constexpr static const size_t bytes_per_element = 16;
-    for (size_t i = 0; i < input.size(); ++i) {
-      uint8_t stack_bytes[bytes_per_element] = {0};
-      uint8_t* bytes = stack_bytes;
-      ToBytes(input[i], &bytes);
-
-      for (size_t i = 0; i < bytes_per_element; ++i) {
-        result.push_back(bytes[i]);
-      }
-    }
-    return result;
-  }
-};
 
-class Decimal32BuilderTest : public ::testing::TestWithParam<int>,
-                             public DecimalTest<Decimal32> {};
+    std::vector<uint8_t> raw_bytes;
 
-class Decimal64BuilderTest : public ::testing::TestWithParam<int>,
-                             public DecimalTest<Decimal64> {};
+    raw_bytes.reserve(size * BYTE_WIDTH);
+    MakeData<BYTE_WIDTH>(draw, &raw_bytes);
 
-class Decimal128BuilderTest : public ::testing::TestWithParam<int>,
-                              public DecimalTest<Decimal128> {};
+    auto expected_data = std::make_shared<Buffer>(raw_bytes.data(), BYTE_WIDTH);
+    std::shared_ptr<Buffer> expected_null_bitmap;
+    ASSERT_OK(BitUtil::BytesToBits(valid_bytes, &expected_null_bitmap));
 
-TEST_P(Decimal32BuilderTest, NoNulls) {
-  int precision = GetParam();
-  std::vector<Decimal32> draw = {Decimal32(1), Decimal32(2), Decimal32(2389),
-                                 Decimal32(4), Decimal32(-12348)};
-  std::vector<uint8_t> valid_bytes = {true, true, true, true, true};
-  this->TestCreate(precision, draw, valid_bytes, 0);
-  this->TestCreate(precision, draw, valid_bytes, 2);
-}
+    int64_t expected_null_count = test::null_count(valid_bytes);
+    auto expected = std::make_shared<DecimalArray>(
+        type, size, expected_data, expected_null_bitmap, expected_null_count);
 
-TEST_P(Decimal64BuilderTest, NoNulls) {
-  int precision = GetParam();
-  std::vector<Decimal64> draw = {Decimal64(1), Decimal64(2), Decimal64(2389),
-                                 Decimal64(4), Decimal64(-12348)};
-  std::vector<uint8_t> valid_bytes = {true, true, true, true, true};
-  this->TestCreate(precision, draw, valid_bytes, 0);
-  this->TestCreate(precision, draw, valid_bytes, 2);
-}
+    std::shared_ptr<Array> lhs = out->Slice(offset);
+    std::shared_ptr<Array> rhs = expected->Slice(offset);
+    bool result = lhs->Equals(rhs);
+    ASSERT_TRUE(result);
+  }
+};
 
-TEST_P(Decimal128BuilderTest, NoNulls) {
+TEST_P(DecimalTest, NoNulls) {
   int precision = GetParam();
-  std::vector<Decimal128> draw = {Decimal128(1), Decimal128(-2), Decimal128(2389),
-                                  Decimal128(4), Decimal128(-12348)};
+  std::vector<Int128> draw = {Int128(1), Int128(-2), Int128(2389), Int128(4),
+                              Int128(-12348)};
   std::vector<uint8_t> valid_bytes = {true, true, true, true, true};
   this->TestCreate(precision, draw, valid_bytes, 0);
   this->TestCreate(precision, draw, valid_bytes, 2);
 }
 
-TEST_P(Decimal32BuilderTest, WithNulls) {
+TEST_P(DecimalTest, WithNulls) {
   int precision = GetParam();
-  std::vector<Decimal32> draw = {Decimal32(1), Decimal32(2), Decimal32(-1), Decimal32(4),
-                                 Decimal32(-1)};
-  std::vector<uint8_t> valid_bytes = {true, true, false, true, false};
-  this->TestCreate(precision, draw, valid_bytes, 0);
-  this->TestCreate(precision, draw, valid_bytes, 2);
-}
+  std::vector<Int128> draw = {Int128(1),  Int128(2), Int128(-1), Int128(4),
+                              Int128(-1), Int128(1), Int128(2)};
+  Int128 big;
+  ASSERT_OK(FromString("230342903942.234234", &big));
+  draw.push_back(big);
 
-TEST_P(Decimal64BuilderTest, WithNulls) {
-  int precision = GetParam();
-  std::vector<Decimal64> draw = {Decimal64(-1), Decimal64(2), Decimal64(-1), Decimal64(4),
-                                 Decimal64(-1)};
-  std::vector<uint8_t> valid_bytes = {true, true, false, true, false};
-  this->TestCreate(precision, draw, valid_bytes, 0);
-  this->TestCreate(precision, draw, valid_bytes, 2);
-}
+  Int128 big_negative;
+  ASSERT_OK(FromString("-23049302932.235234", &big_negative));
+  draw.push_back(big_negative);
 
-TEST_P(Decimal128BuilderTest, WithNulls) {
-  int precision = GetParam();
-  std::vector<Decimal128> draw = {Decimal128(1),
-                                  Decimal128(2),
-                                  Decimal128(-1),
-                                  Decimal128(4),
-                                  Decimal128(-1),
-                                  Decimal128(1),
-                                  Decimal128(2),
-                                  Decimal128("230342903942.234234"),
-                                  Decimal128("-23049302932.235234")};
   std::vector<uint8_t> valid_bytes = {true, true, false, true, false,
                                       true, true, true,  true};
   this->TestCreate(precision, draw, valid_bytes, 0);
   this->TestCreate(precision, draw, valid_bytes, 2);
 }
 
-INSTANTIATE_TEST_CASE_P(Decimal32BuilderTest, Decimal32BuilderTest,
-                        ::testing::Range(DecimalPrecision<int32_t>::minimum,
-                                         DecimalPrecision<int32_t>::maximum));
-INSTANTIATE_TEST_CASE_P(Decimal64BuilderTest, Decimal64BuilderTest,
-                        ::testing::Range(DecimalPrecision<int64_t>::minimum,
-                                         DecimalPrecision<int64_t>::maximum));
-INSTANTIATE_TEST_CASE_P(Decimal128BuilderTest, Decimal128BuilderTest,
-                        ::testing::Range(DecimalPrecision<Int128>::minimum,
-                                         DecimalPrecision<Int128>::maximum));
+INSTANTIATE_TEST_CASE_P(DecimalTest, DecimalTest, ::testing::Range(1, 38));
 
 }  // namespace decimal
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/array.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index 720d5ec..babf326 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -29,6 +29,7 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/bit-util.h"
 #include "arrow/util/decimal.h"
+#include "arrow/util/int128.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
 #include "arrow/visitor.h"
@@ -316,28 +317,10 @@ DecimalArray::DecimalArray(const std::shared_ptr<internal::ArrayData>& data)
   DCHECK_EQ(data->type->id(), Type::DECIMAL);
 }
 
-#define DECIMAL_TO_STRING_CASE(bits, bytes, precision, scale) \
-  case bits: {                                                \
-    decimal::Decimal##bits value;                             \
-    decimal::FromBytes((bytes), &value);                      \
-    return decimal::ToString(value, (precision), (scale));    \
-  }
-
 std::string DecimalArray::FormatValue(int64_t i) const {
   const auto& type_ = static_cast<const DecimalType&>(*type());
-  const int precision = type_.precision();
-  const int scale = type_.scale();
-  const int bit_width = type_.bit_width();
-  const uint8_t* bytes = GetValue(i);
-  switch (bit_width) {
-    DECIMAL_TO_STRING_CASE(32, bytes, precision, scale)
-    DECIMAL_TO_STRING_CASE(64, bytes, precision, scale)
-    DECIMAL_TO_STRING_CASE(128, bytes, precision, scale)
-    default: {
-      DCHECK(false) << "Invalid bit width: " << bit_width;
-      return "";
-    }
-  }
+  decimal::Int128 value(GetValue(i));
+  return decimal::ToString(value, type_.precision(), type_.scale());
 }
 
 // ----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/buffer.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h
index d5b2a26..4c3bd79 100644
--- a/cpp/src/arrow/buffer.h
+++ b/cpp/src/arrow/buffer.h
@@ -18,6 +18,7 @@
 #ifndef ARROW_BUFFER_H
 #define ARROW_BUFFER_H
 
+#include <algorithm>
 #include <cstdint>
 #include <cstring>
 #include <memory>
@@ -215,6 +216,19 @@ class ARROW_EXPORT BufferBuilder {
     return Status::OK();
   }
 
+  template <size_t NBYTES>
+  Status Append(const std::array<uint8_t, NBYTES>& data) {
+    constexpr auto nbytes = static_cast<int64_t>(NBYTES);
+    if (capacity_ < static_cast<int64_t>(nbytes) + size_) {
+      int64_t new_capacity = BitUtil::NextPower2(nbytes + size_);
+      RETURN_NOT_OK(Resize(new_capacity));
+    }
+
+    std::copy(data.cbegin(), data.cend(), data_ + size_);
+    size_ += nbytes;
+    return Status::OK();
+  }
+
   // Advance pointer and zero out memory
   Status Advance(const int64_t length) {
     if (capacity_ < length + size_) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/builder.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.cc b/cpp/src/arrow/builder.cc
index 71f0bab..a1d2366 100644
--- a/cpp/src/arrow/builder.cc
+++ b/cpp/src/arrow/builder.cc
@@ -34,6 +34,7 @@
 #include "arrow/util/cpu-info.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/hash-util.h"
+#include "arrow/util/int128.h"
 #include "arrow/util/logging.h"
 
 namespace arrow {
@@ -965,31 +966,17 @@ DecimalBuilder::DecimalBuilder(MemoryPool* pool, const std::shared_ptr<DataType>
     : DecimalBuilder(type, pool) {}
 #endif
 
-template <typename T>
-ARROW_EXPORT Status DecimalBuilder::Append(const decimal::Decimal<T>& val) {
-  RETURN_NOT_OK(FixedSizeBinaryBuilder::Reserve(1));
-  return FixedSizeBinaryBuilder::Append(reinterpret_cast<const uint8_t*>(&val.value));
-}
-
-template ARROW_EXPORT Status DecimalBuilder::Append(const decimal::Decimal32& val);
-template ARROW_EXPORT Status DecimalBuilder::Append(const decimal::Decimal64& val);
-
-template <>
-ARROW_EXPORT Status DecimalBuilder::Append(const decimal::Decimal128& value) {
+Status DecimalBuilder::Append(const decimal::Int128& value) {
   RETURN_NOT_OK(FixedSizeBinaryBuilder::Reserve(1));
-  uint8_t stack_bytes[16] = {0};
-  uint8_t* bytes = stack_bytes;
-  decimal::ToBytes(value, &bytes);
+  std::array<uint8_t, 16> bytes;
+  RETURN_NOT_OK(value.ToBytes(&bytes));
   return FixedSizeBinaryBuilder::Append(bytes);
 }
 
 Status DecimalBuilder::Finish(std::shared_ptr<Array>* out) {
   std::shared_ptr<Buffer> data;
   RETURN_NOT_OK(byte_builder_.Finish(&data));
-
-  /// TODO(phillipc): not sure where to get the offset argument here
-  *out =
-      std::make_shared<DecimalArray>(type_, length_, data, null_bitmap_, null_count_, 0);
+  *out = std::make_shared<DecimalArray>(type_, length_, data, null_bitmap_, null_count_);
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/builder.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/builder.h b/cpp/src/arrow/builder.h
index cb1e4ba..d1b51d6 100644
--- a/cpp/src/arrow/builder.h
+++ b/cpp/src/arrow/builder.h
@@ -47,8 +47,7 @@ struct ArrayData;
 
 namespace decimal {
 
-template <typename T>
-struct Decimal;
+class Int128;
 
 }  // namespace decimal
 
@@ -674,6 +673,14 @@ class ARROW_EXPORT FixedSizeBinaryBuilder : public ArrayBuilder {
                          MemoryPool* pool ARROW_MEMORY_POOL_DEFAULT);
 
   Status Append(const uint8_t* value);
+
+  template <size_t NBYTES>
+  Status Append(const std::array<uint8_t, NBYTES>& value) {
+    RETURN_NOT_OK(Reserve(1));
+    UnsafeAppendToBitmap(true);
+    return byte_builder_.Append(value);
+  }
+
   Status Append(const uint8_t* data, int64_t length,
                 const uint8_t* valid_bytes = nullptr);
   Status Append(const std::string& value);
@@ -701,8 +708,7 @@ class ARROW_EXPORT DecimalBuilder : public FixedSizeBinaryBuilder {
   explicit DecimalBuilder(MemoryPool* pool, const std::shared_ptr<DataType>& type);
 #endif
 
-  template <typename T>
-  ARROW_EXPORT Status Append(const decimal::Decimal<T>& val);
+  Status Append(const decimal::Int128& val);
 
   Status Finish(std::shared_ptr<Array>* out) override;
 };

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/compare.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/compare.cc b/cpp/src/arrow/compare.cc
index 1b84603..b78d5d5 100644
--- a/cpp/src/arrow/compare.cc
+++ b/cpp/src/arrow/compare.cc
@@ -313,20 +313,14 @@ static bool IsEqualPrimitive(const PrimitiveArray& left, const PrimitiveArray& r
   const auto& size_meta = dynamic_cast<const FixedWidthType&>(*left.type());
   const int byte_width = size_meta.bit_width() / CHAR_BIT;
 
-  const uint8_t* left_data = nullptr;
-  const uint8_t* right_data = nullptr;
-
-  if (left.values()) {
-    left_data = left.raw_values();
-  }
-  if (right.values()) {
-    right_data = right.raw_values();
-  }
+  const uint8_t* left_data = left.values() ? left.raw_values() : nullptr;
+  const uint8_t* right_data = right.values() ? right.raw_values() : nullptr;
 
   if (left.null_count() > 0) {
     for (int64_t i = 0; i < left.length(); ++i) {
-      bool left_null = left.IsNull(i);
-      if (!left_null && (memcmp(left_data, right_data, byte_width) || right.IsNull(i))) {
+      const bool left_null = left.IsNull(i);
+      const bool right_null = right.IsNull(i);
+      if (!left_null && (memcmp(left_data, right_data, byte_width) != 0 || right_null)) {
         return false;
       }
       left_data += byte_width;
@@ -334,42 +328,18 @@ static bool IsEqualPrimitive(const PrimitiveArray& left, const PrimitiveArray& r
     }
     return true;
   } else {
-    return memcmp(left_data, right_data,
-                  static_cast<size_t>(byte_width * left.length())) == 0;
+    auto number_of_bytes_to_compare = static_cast<size_t>(byte_width * left.length());
+    return memcmp(left_data, right_data, number_of_bytes_to_compare) == 0;
   }
 }
 
-template <typename T>
-static inline bool CompareBuiltIn(const Array& left, const Array& right, const T* ldata,
-                                  const T* rdata) {
-  if (ldata == nullptr && rdata == nullptr) {
-    return true;
-  }
-
-  if (ldata == nullptr || rdata == nullptr) {
-    return false;
-  }
-
-  if (left.null_count() > 0) {
-    for (int64_t i = 0; i < left.length(); ++i) {
-      if (left.IsNull(i) != right.IsNull(i)) {
-        return false;
-      } else if (!left.IsNull(i) && (ldata[i] != rdata[i])) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  return memcmp(ldata, rdata, sizeof(T) * left.length()) == 0;
-}
-
 class ArrayEqualsVisitor : public RangeEqualsVisitor {
  public:
   explicit ArrayEqualsVisitor(const Array& right)
       : RangeEqualsVisitor(right, 0, right.length(), 0) {}
 
   Status Visit(const NullArray& left) {
+    ARROW_UNUSED(left);
     result_ = true;
     return Status::OK();
   }
@@ -507,27 +477,6 @@ class ArrayEqualsVisitor : public RangeEqualsVisitor {
     return Status::OK();
   }
 
-  Status Visit(const DecimalArray& left) {
-    const int byte_width = left.byte_width();
-    if (byte_width == 4) {
-      result_ = CompareBuiltIn<int32_t>(
-          left, right_, reinterpret_cast<const int32_t*>(left.raw_values()),
-          reinterpret_cast<const int32_t*>(
-              static_cast<const DecimalArray&>(right_).raw_values()));
-      return Status::OK();
-    }
-
-    if (byte_width == 8) {
-      result_ = CompareBuiltIn<int64_t>(
-          left, right_, reinterpret_cast<const int64_t*>(left.raw_values()),
-          reinterpret_cast<const int64_t*>(
-              static_cast<const DecimalArray&>(right_).raw_values()));
-      return Status::OK();
-    }
-
-    return RangeEqualsVisitor::Visit(left);
-  }
-
   template <typename T>
   typename std::enable_if<std::is_base_of<NestedType, typename T::TypeClass>::value,
                           Status>::type

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/python/arrow_to_pandas.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/arrow_to_pandas.cc b/cpp/src/arrow/python/arrow_to_pandas.cc
index 8ba3f46..d2d6fb4 100644
--- a/cpp/src/arrow/python/arrow_to_pandas.cc
+++ b/cpp/src/arrow/python/arrow_to_pandas.cc
@@ -37,6 +37,7 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/bit-util.h"
 #include "arrow/util/decimal.h"
+#include "arrow/util/int128.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
 #include "arrow/util/parallel.h"
@@ -602,44 +603,14 @@ static Status ConvertTimes(PandasOptions options, const ChunkedArray& data,
   return Status::OK();
 }
 
-template <typename T>
-Status ValidateDecimalPrecision(int precision) {
-  constexpr static const int kMaximumPrecision =
-      decimal::DecimalPrecision<typename T::value_type>::maximum;
-  if (!(precision > 0 && precision <= kMaximumPrecision)) {
-    std::stringstream ss;
-    ss << "Invalid precision: " << precision << ". Minimum is 1, maximum is "
-       << kMaximumPrecision;
-    return Status::Invalid(ss.str());
-  }
-  return Status::OK();
-}
-
-template <typename T>
-Status RawDecimalToString(const uint8_t* bytes, int precision, int scale,
-                          std::string* result) {
-  DCHECK_NE(bytes, nullptr);
+static Status RawDecimalToString(const uint8_t* bytes, int precision, int scale,
+                                 std::string* result) {
   DCHECK_NE(result, nullptr);
-  RETURN_NOT_OK(ValidateDecimalPrecision<T>(precision));
-  T decimal;
-  decimal::FromBytes(bytes, &decimal);
+  decimal::Int128 decimal(bytes);
   *result = decimal::ToString(decimal, precision, scale);
   return Status::OK();
 }
 
-template Status RawDecimalToString<decimal::Decimal32>(const uint8_t*, int, int,
-                                                       std::string*);
-template Status RawDecimalToString<decimal::Decimal64>(const uint8_t*, int, int,
-                                                       std::string*);
-template Status RawDecimalToString<decimal::Decimal128>(const uint8_t*, int, int,
-                                                        std::string*);
-
-#define RAW_DECIMAL_TO_STRING_CASE(bits, value, precision, scale, output)          \
-  case bits:                                                                       \
-    RETURN_NOT_OK(RawDecimalToString<decimal::Decimal##bits>((value), (precision), \
-                                                             (scale), (output)));  \
-    break;
-
 static Status ConvertDecimals(PandasOptions options, const ChunkedArray& data,
                               PyObject** out_values) {
   PyAcquireGIL lock;
@@ -654,7 +625,6 @@ static Status ConvertDecimals(PandasOptions options, const ChunkedArray& data,
     auto type(std::dynamic_pointer_cast<arrow::DecimalType>(arr->type()));
     const int precision = type->precision();
     const int scale = type->scale();
-    const int bit_width = type->bit_width();
 
     for (int64_t i = 0; i < arr->length(); ++i) {
       if (arr->IsNull(i)) {
@@ -663,16 +633,7 @@ static Status ConvertDecimals(PandasOptions options, const ChunkedArray& data,
       } else {
         const uint8_t* raw_value = arr->GetValue(i);
         std::string decimal_string;
-        switch (bit_width) {
-          RAW_DECIMAL_TO_STRING_CASE(32, raw_value, precision, scale, &decimal_string)
-          RAW_DECIMAL_TO_STRING_CASE(64, raw_value, precision, scale, &decimal_string)
-          RAW_DECIMAL_TO_STRING_CASE(128, raw_value, precision, scale, &decimal_string)
-          default: {
-            std::stringstream buf;
-            buf << "Invalid bit_width " << bit_width << " for decimal value";
-            return Status::Invalid(buf.str());
-          }
-        }
+        RETURN_NOT_OK(RawDecimalToString(raw_value, precision, scale, &decimal_string));
         RETURN_NOT_OK(DecimalFromString(Decimal, decimal_string, out_values++));
       }
     }

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/python/builtin_convert.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/builtin_convert.cc b/cpp/src/arrow/python/builtin_convert.cc
index ccaf280..dca3f45 100644
--- a/cpp/src/arrow/python/builtin_convert.cc
+++ b/cpp/src/arrow/python/builtin_convert.cc
@@ -28,6 +28,7 @@
 #include "arrow/api.h"
 #include "arrow/status.h"
 #include "arrow/util/decimal.h"
+#include "arrow/util/int128.h"
 #include "arrow/util/logging.h"
 
 #include "arrow/python/helpers.h"
@@ -579,44 +580,24 @@ class ListConverter : public TypedConverterVisitor<ListBuilder, ListConverter> {
   std::shared_ptr<SeqConverter> value_converter_;
 };
 
-#define DECIMAL_CONVERT_CASE(bit_width, item, builder)         \
-  case bit_width: {                                            \
-    arrow::decimal::Decimal##bit_width out;                    \
-    std::string string_out;                                    \
-    RETURN_NOT_OK(PythonDecimalToString((item), &string_out)); \
-    RETURN_NOT_OK(FromString(string_out, &out));               \
-    return ((builder)->Append(out));                           \
-    break;                                                     \
-  }
-
 class DecimalConverter
     : public TypedConverterVisitor<arrow::DecimalBuilder, DecimalConverter> {
  public:
   inline Status AppendItem(const OwnedRef& item) {
-    /// Can the compiler figure out that the case statement below isn't necessary
-    /// once we're running?
-    const int bit_width =
-        std::dynamic_pointer_cast<arrow::DecimalType>(typed_builder_->type())
-            ->bit_width();
-
     /// TODO(phillipc): Check for nan?
     if (item.obj() != Py_None) {
-      switch (bit_width) {
-        DECIMAL_CONVERT_CASE(32, item.obj(), typed_builder_)
-        DECIMAL_CONVERT_CASE(64, item.obj(), typed_builder_)
-        DECIMAL_CONVERT_CASE(128, item.obj(), typed_builder_)
-        default:
-          return Status::OK();
-      }
-      RETURN_IF_PYERROR();
-    } else {
-      return typed_builder_->AppendNull();
+      std::string string;
+      RETURN_NOT_OK(PythonDecimalToString(item.obj(), &string));
+
+      decimal::Int128 value;
+      RETURN_NOT_OK(decimal::FromString(string, &value));
+      return typed_builder_->Append(value);
     }
+
+    return typed_builder_->AppendNull();
   }
 };
 
-#undef DECIMAL_CONVERT_CASE
-
 // Dynamic constructor for sequence converters
 std::shared_ptr<SeqConverter> GetConverter(const std::shared_ptr<DataType>& type) {
   switch (type->id()) {

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/python/helpers.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/helpers.cc b/cpp/src/arrow/python/helpers.cc
index 164e42e..67f0655 100644
--- a/cpp/src/arrow/python/helpers.cc
+++ b/cpp/src/arrow/python/helpers.cc
@@ -18,6 +18,7 @@
 #include "arrow/python/helpers.h"
 #include "arrow/python/common.h"
 #include "arrow/util/decimal.h"
+#include "arrow/util/logging.h"
 
 #include <arrow/api.h>
 
@@ -102,8 +103,7 @@ Status InferDecimalPrecisionAndScale(PyObject* python_decimal, int* precision,
   auto size = str.size;
 
   std::string c_string(bytes, size);
-  return FromString(c_string, static_cast<decimal::Decimal32*>(nullptr), precision,
-                    scale);
+  return decimal::FromString(c_string, nullptr, precision, scale);
 }
 
 Status DecimalFromString(PyObject* decimal_constructor, const std::string& decimal_string,

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/python/pandas_to_arrow.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/pandas_to_arrow.cc b/cpp/src/arrow/python/pandas_to_arrow.cc
index 435682c..fe8ae80 100644
--- a/cpp/src/arrow/python/pandas_to_arrow.cc
+++ b/cpp/src/arrow/python/pandas_to_arrow.cc
@@ -39,6 +39,7 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/bit-util.h"
 #include "arrow/util/decimal.h"
+#include "arrow/util/int128.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
 #include "arrow/visitor_inline.h"
@@ -605,16 +606,6 @@ Status PandasConverter::ConvertDates() {
   return PushBuilderResult(&builder);
 }
 
-#define CONVERT_DECIMAL_CASE(bit_width, builder, object)         \
-  case bit_width: {                                              \
-    decimal::Decimal##bit_width d;                               \
-    std::string string_out;                                      \
-    RETURN_NOT_OK(PythonDecimalToString((object), &string_out)); \
-    RETURN_NOT_OK(FromString(string_out, &d));                   \
-    RETURN_NOT_OK((builder).Append(d));                          \
-    break;                                                       \
-  }
-
 Status PandasConverter::ConvertDecimals() {
   PyAcquireGIL lock;
 
@@ -634,20 +625,18 @@ Status PandasConverter::ConvertDecimals() {
 
   type_ = std::make_shared<DecimalType>(precision, scale);
 
-  const int bit_width = std::dynamic_pointer_cast<DecimalType>(type_)->bit_width();
   DecimalBuilder builder(type_, pool_);
   RETURN_NOT_OK(builder.Resize(length_));
 
   for (int64_t i = 0; i < length_; ++i) {
     object = objects[i];
     if (PyObject_IsInstance(object, Decimal.obj())) {
-      switch (bit_width) {
-        CONVERT_DECIMAL_CASE(32, builder, object)
-        CONVERT_DECIMAL_CASE(64, builder, object)
-        CONVERT_DECIMAL_CASE(128, builder, object)
-        default:
-          break;
-      }
+      std::string string;
+      RETURN_NOT_OK(PythonDecimalToString(object, &string));
+
+      decimal::Int128 value;
+      RETURN_NOT_OK(decimal::FromString(string, &value));
+      RETURN_NOT_OK(builder.Append(value));
     } else if (PandasObjectIsNull(object)) {
       RETURN_NOT_OK(builder.AppendNull());
     } else {
@@ -688,8 +677,6 @@ Status PandasConverter::ConvertTimes() {
   return PushBuilderResult(&builder);
 }
 
-#undef CONVERT_DECIMAL_CASE
-
 Status PandasConverter::ConvertObjectStrings() {
   PyAcquireGIL lock;
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/type-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type-test.cc b/cpp/src/arrow/type-test.cc
index 138f8dd..710344a 100644
--- a/cpp/src/arrow/type-test.cc
+++ b/cpp/src/arrow/type-test.cc
@@ -401,7 +401,7 @@ TEST(TestStructType, Basics) {
   // TODO(wesm): out of bounds for field(...)
 }
 
-TEST(TypesTest, TestDecimal32Type) {
+TEST(TypesTest, TestDecimal128Small) {
   DecimalType t1(8, 4);
 
   ASSERT_EQ(t1.id(), Type::DECIMAL);
@@ -411,11 +411,11 @@ TEST(TypesTest, TestDecimal32Type) {
   ASSERT_EQ(t1.ToString(), std::string("decimal(8, 4)"));
 
   // Test properties
-  ASSERT_EQ(t1.byte_width(), 4);
-  ASSERT_EQ(t1.bit_width(), 32);
+  ASSERT_EQ(t1.byte_width(), 16);
+  ASSERT_EQ(t1.bit_width(), 128);
 }
 
-TEST(TypesTest, TestDecimal64Type) {
+TEST(TypesTest, TestDecimal128Medium) {
   DecimalType t1(12, 5);
 
   ASSERT_EQ(t1.id(), Type::DECIMAL);
@@ -425,11 +425,11 @@ TEST(TypesTest, TestDecimal64Type) {
   ASSERT_EQ(t1.ToString(), std::string("decimal(12, 5)"));
 
   // Test properties
-  ASSERT_EQ(t1.byte_width(), 8);
-  ASSERT_EQ(t1.bit_width(), 64);
+  ASSERT_EQ(t1.byte_width(), 16);
+  ASSERT_EQ(t1.bit_width(), 128);
 }
 
-TEST(TypesTest, TestDecimal128Type) {
+TEST(TypesTest, TestDecimal128Large) {
   DecimalType t1(27, 7);
 
   ASSERT_EQ(t1.id(), Type::DECIMAL);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index aeb6130..3b7ef19 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -454,25 +454,12 @@ class ARROW_EXPORT StructType : public NestedType {
   std::vector<BufferDescr> GetBufferLayout() const override;
 };
 
-static inline int decimal_byte_width(int precision) {
-  if (precision >= 0 && precision < 10) {
-    return 4;
-  } else if (precision >= 10 && precision < 19) {
-    return 8;
-  } else {
-    // TODO(phillipc): validate that we can't construct > 128 bit types
-    return 16;
-  }
-}
-
 class ARROW_EXPORT DecimalType : public FixedSizeBinaryType {
  public:
   static constexpr Type::type type_id = Type::DECIMAL;
 
   explicit DecimalType(int precision, int scale)
-      : FixedSizeBinaryType(decimal_byte_width(precision), Type::DECIMAL),
-        precision_(precision),
-        scale_(scale) {}
+      : FixedSizeBinaryType(16, Type::DECIMAL), precision_(precision), scale_(scale) {}
 
   Status Accept(TypeVisitor* visitor) const override;
   std::string ToString() const override;

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/util/decimal-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/decimal-test.cc b/cpp/src/arrow/util/decimal-test.cc
index ea8962b..5162971 100644
--- a/cpp/src/arrow/util/decimal-test.cc
+++ b/cpp/src/arrow/util/decimal-test.cc
@@ -24,76 +24,73 @@
 #include "arrow/status.h"
 #include "arrow/test-util.h"
 #include "arrow/util/decimal.h"
+#include "arrow/util/int128.h"
 
 namespace arrow {
 namespace decimal {
 
-template <typename T>
-class DecimalTest : public ::testing::Test {
+class DecimalTestFixture : public ::testing::Test {
  public:
-  DecimalTest() : decimal_value_(23423445), string_value_("234.23445") {}
-  Decimal<T> decimal_value_;
+  DecimalTestFixture() : integer_value_(23423445), string_value_("234.23445") {}
+  Int128 integer_value_;
   std::string string_value_;
 };
 
-typedef ::testing::Types<int32_t, int64_t, Int128> DecimalTypes;
-TYPED_TEST_CASE(DecimalTest, DecimalTypes);
-
-TYPED_TEST(DecimalTest, TestToString) {
-  Decimal<TypeParam> decimal(this->decimal_value_);
+TEST_F(DecimalTestFixture, TestToString) {
+  Int128 decimal(this->integer_value_);
   int precision = 8;
   int scale = 5;
   std::string result = ToString(decimal, precision, scale);
   ASSERT_EQ(result, this->string_value_);
 }
 
-TYPED_TEST(DecimalTest, TestFromString) {
-  Decimal<TypeParam> expected(this->decimal_value_);
-  Decimal<TypeParam> result;
+TEST_F(DecimalTestFixture, TestFromString) {
+  Int128 expected(this->integer_value_);
+  Int128 result;
   int precision, scale;
   ASSERT_OK(FromString(this->string_value_, &result, &precision, &scale));
-  ASSERT_EQ(result.value, expected.value);
+  ASSERT_EQ(result, expected);
   ASSERT_EQ(precision, 8);
   ASSERT_EQ(scale, 5);
 }
 
-TEST(DecimalTest, TestStringStartingWithPlus) {
+TEST_F(DecimalTestFixture, TestStringStartingWithPlus) {
   std::string plus_value("+234.234");
-  Decimal32 out;
+  Int128 out;
   int scale;
   int precision;
   ASSERT_OK(FromString(plus_value, &out, &precision, &scale));
-  ASSERT_EQ(234234, out.value);
+  ASSERT_EQ(234234, out);
   ASSERT_EQ(6, precision);
   ASSERT_EQ(3, scale);
 }
 
-TEST(DecimalTest, TestStringStartingWithPlus128) {
+TEST_F(DecimalTestFixture, TestStringStartingWithPlus128) {
   std::string plus_value("+2342394230592.232349023094");
   Int128 expected_value("2342394230592232349023094");
-  Decimal128 out;
+  Int128 out;
   int scale;
   int precision;
   ASSERT_OK(FromString(plus_value, &out, &precision, &scale));
-  ASSERT_EQ(expected_value, out.value);
+  ASSERT_EQ(expected_value, out);
   ASSERT_EQ(25, precision);
   ASSERT_EQ(12, scale);
 }
 
 TEST(DecimalTest, TestStringToInt32) {
-  int32_t value = 0;
+  Int128 value;
   StringToInteger("123", "456", 1, &value);
   ASSERT_EQ(value, 123456);
 }
 
 TEST(DecimalTest, TestStringToInt64) {
-  int64_t value = 0;
+  Int128 value;
   StringToInteger("123456789", "456", -1, &value);
   ASSERT_EQ(value, -123456789456);
 }
 
 TEST(DecimalTest, TestStringToInt128) {
-  Int128 value = 0;
+  Int128 value;
   StringToInteger("123456789", "456789123", 1, &value);
   ASSERT_EQ(value.high_bits(), 0);
   ASSERT_EQ(value.low_bits(), 123456789456789123);
@@ -101,120 +98,99 @@ TEST(DecimalTest, TestStringToInt128) {
 
 TEST(DecimalTest, TestFromString128) {
   static const std::string string_value("-23049223942343532412");
-  Decimal128 result(string_value);
+  Int128 result(string_value);
   Int128 expected(static_cast<int64_t>(-230492239423435324));
-  ASSERT_EQ(result.value, expected * 100 - 12);
+  ASSERT_EQ(result, expected * 100 - 12);
 
   // Sanity check that our number is actually using more than 64 bits
-  ASSERT_NE(result.value.high_bits(), 0);
+  ASSERT_NE(result.high_bits(), 0);
 }
 
 TEST(DecimalTest, TestFromDecimalString128) {
-  static const std::string string_value("-23049223942343.532412");
-  Decimal128 result(string_value);
+  std::string string_value("-23049223942343.532412");
+  Int128 result;
+  ASSERT_OK(FromString(string_value, &result));
   Int128 expected(static_cast<int64_t>(-230492239423435324));
-  ASSERT_EQ(result.value, expected * 100 - 12);
+  expected *= 100;
+  expected -= 12;
+  ASSERT_EQ(result, expected);
 
   // Sanity check that our number is actually using more than 64 bits
-  ASSERT_NE(result.value.high_bits(), 0);
-}
-
-TEST(DecimalTest, TestDecimal32Precision) {
-  auto min_precision = DecimalPrecision<int32_t>::minimum;
-  auto max_precision = DecimalPrecision<int32_t>::maximum;
-  ASSERT_EQ(min_precision, 1);
-  ASSERT_EQ(max_precision, 9);
-}
-
-TEST(DecimalTest, TestDecimal64Precision) {
-  auto min_precision = DecimalPrecision<int64_t>::minimum;
-  auto max_precision = DecimalPrecision<int64_t>::maximum;
-  ASSERT_EQ(min_precision, 10);
-  ASSERT_EQ(max_precision, 18);
-}
-
-TEST(DecimalTest, TestDecimal128Precision) {
-  auto min_precision = DecimalPrecision<Int128>::minimum;
-  auto max_precision = DecimalPrecision<Int128>::maximum;
-  ASSERT_EQ(min_precision, 19);
-  ASSERT_EQ(max_precision, 38);
+  ASSERT_NE(result.high_bits(), 0);
 }
 
 TEST(DecimalTest, TestDecimal32SignedRoundTrip) {
-  Decimal32 expected(std::string("-3402692"));
+  Int128 expected("-3402692");
 
-  uint8_t stack_bytes[4] = {0};
-  uint8_t* bytes = stack_bytes;
-  ToBytes(expected, &bytes);
+  std::array<uint8_t, 16> bytes;
+  ASSERT_OK(expected.ToBytes(&bytes));
 
-  Decimal32 result;
-  FromBytes(bytes, &result);
-  ASSERT_EQ(expected.value, result.value);
+  Int128 result(bytes.data());
+  ASSERT_EQ(expected, result);
 }
 
 TEST(DecimalTest, TestDecimal64SignedRoundTrip) {
-  Decimal64 expected(std::string("-34034293045.921"));
+  Int128 expected;
+  std::string string_value("-34034293045.921");
+  ASSERT_OK(FromString(string_value, &expected));
 
-  uint8_t stack_bytes[8] = {0};
-  uint8_t* bytes = stack_bytes;
-  ToBytes(expected, &bytes);
+  std::array<uint8_t, 16> bytes;
+  ASSERT_OK(expected.ToBytes(&bytes));
 
-  Decimal64 result;
-  FromBytes(bytes, &result);
+  Int128 result(bytes.data());
 
-  ASSERT_EQ(expected.value, result.value);
+  ASSERT_EQ(expected, result);
 }
 
-TEST(DecimalTest, TestDecimal128StringAndBytesRoundTrip) {
+TEST(DecimalTest, TestDecimalStringAndBytesRoundTrip) {
+  Int128 expected;
   std::string string_value("-340282366920938463463374607431.711455");
-  Decimal128 expected(string_value);
+  ASSERT_OK(FromString(string_value, &expected));
 
   std::string expected_string_value("-340282366920938463463374607431711455");
   Int128 expected_underlying_value(expected_string_value);
 
-  ASSERT_EQ(expected.value, expected_underlying_value);
+  ASSERT_EQ(expected, expected_underlying_value);
 
-  uint8_t stack_bytes[16] = {0};
-  uint8_t* bytes = stack_bytes;
-  ToBytes(expected, &bytes);
+  std::array<uint8_t, 16> bytes;
+  ASSERT_OK(expected.ToBytes(&bytes));
 
-  Decimal128 result;
-  FromBytes(bytes, &result);
+  Int128 result(bytes.data());
 
-  ASSERT_EQ(expected.value, result.value);
+  ASSERT_EQ(expected, result);
 }
 
 TEST(DecimalTest, TestInvalidInputMinus) {
   std::string invalid_value("-");
-  Decimal32 out;
+  Int128 out;
   Status status = decimal::FromString(invalid_value, &out);
   ASSERT_RAISES(Invalid, status);
 }
 
 TEST(DecimalTest, TestInvalidInputDot) {
   std::string invalid_value("0.0.0");
-  Decimal32 out;
+  Int128 out;
   Status status = decimal::FromString(invalid_value, &out);
   ASSERT_RAISES(Invalid, status);
 }
 
 TEST(DecimalTest, TestInvalidInputEmbeddedMinus) {
   std::string invalid_value("0-13-32");
-  Decimal32 out;
+  Int128 out;
   Status status = decimal::FromString(invalid_value, &out);
   ASSERT_RAISES(Invalid, status);
 }
 
 TEST(DecimalTest, TestInvalidInputSingleChar) {
   std::string invalid_value("a");
-  Decimal32 out;
+  Int128 out;
   Status status = decimal::FromString(invalid_value, &out);
   ASSERT_RAISES(Invalid, status);
 }
 
 TEST(DecimalTest, TestInvalidInputWithValidSubstring) {
   std::string invalid_value("-23092.235-");
-  Decimal32 out;
+  Int128 out;
   Status status = decimal::FromString(invalid_value, &out);
   auto msg = status.message();
   ASSERT_RAISES(Invalid, status);
@@ -222,43 +198,39 @@ TEST(DecimalTest, TestInvalidInputWithValidSubstring) {
 
 TEST(DecimalTest, TestInvalidInputWithMinusPlus) {
   std::string invalid_value("-+23092.235");
-  Decimal32 out;
+  Int128 out;
   Status status = decimal::FromString(invalid_value, &out);
   ASSERT_RAISES(Invalid, status);
 }
 
 TEST(DecimalTest, TestInvalidInputWithPlusMinus) {
   std::string invalid_value("+-23092.235");
-  Decimal32 out;
+  Int128 out;
   Status status = decimal::FromString(invalid_value, &out);
   ASSERT_RAISES(Invalid, status);
 }
 
 TEST(DecimalTest, TestInvalidInputWithLeadingZeros) {
   std::string invalid_value("00a");
-  Decimal32 out;
+  Int128 out;
   Status status = decimal::FromString(invalid_value, &out);
   ASSERT_RAISES(Invalid, status);
 }
 
-template <typename T>
-class DecimalZerosTest : public ::testing::Test {};
-TYPED_TEST_CASE(DecimalZerosTest, DecimalTypes);
-
-TYPED_TEST(DecimalZerosTest, LeadingZerosNoDecimalPoint) {
+TEST(DecimalZerosTest, LeadingZerosNoDecimalPoint) {
   std::string string_value("0000000");
-  Decimal<TypeParam> d;
+  Int128 d;
   int precision;
   int scale;
   ASSERT_OK(FromString(string_value, &d, &precision, &scale));
   ASSERT_EQ(precision, 7);
   ASSERT_EQ(scale, 0);
-  ASSERT_EQ(d.value, 0);
+  ASSERT_EQ(d, 0);
 }
 
-TYPED_TEST(DecimalZerosTest, LeadingZerosDecimalPoint) {
+TEST(DecimalZerosTest, LeadingZerosDecimalPoint) {
   std::string string_value("000.0000");
-  Decimal<TypeParam> d;
+  Int128 d;
   int precision;
   int scale;
   ASSERT_OK(FromString(string_value, &d, &precision, &scale));
@@ -266,18 +238,48 @@ TYPED_TEST(DecimalZerosTest, LeadingZerosDecimalPoint) {
   ASSERT_NE(precision, 7);
 
   ASSERT_EQ(scale, 4);
-  ASSERT_EQ(d.value, 0);
+  ASSERT_EQ(d, 0);
 }
 
-TYPED_TEST(DecimalZerosTest, NoLeadingZerosDecimalPoint) {
+TEST(DecimalZerosTest, NoLeadingZerosDecimalPoint) {
   std::string string_value(".00000");
-  Decimal<TypeParam> d;
+  Int128 d;
   int precision;
   int scale;
   ASSERT_OK(FromString(string_value, &d, &precision, &scale));
   ASSERT_EQ(precision, 5);
   ASSERT_EQ(scale, 5);
-  ASSERT_EQ(d.value, 0);
+  ASSERT_EQ(d, 0);
+}
+
+template <typename T>
+class Int128Test : public ::testing::Test {
+ public:
+  Int128Test() : value_(42) {}
+  const T value_;
+};
+
+using Int128Types =
+    ::testing::Types<char, unsigned char, short, unsigned short,  // NOLINT
+                     int, unsigned int, long, unsigned long,      // NOLINT
+                     long long, unsigned long long                // NOLINT
+                     >;
+
+TYPED_TEST_CASE(Int128Test, Int128Types);
+
+TYPED_TEST(Int128Test, ConstructibleFromAnyIntegerType) {
+  Int128 value(this->value_);
+  ASSERT_EQ(42, value.low_bits());
+}
+
+TEST(Int128TestTrue, ConstructibleFromBool) {
+  Int128 value(true);
+  ASSERT_EQ(1, value.low_bits());
+}
+
+TEST(Int128TestFalse, ConstructibleFromBool) {
+  Int128 value(false);
+  ASSERT_EQ(0, value.low_bits());
 }
 
 }  // namespace decimal

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/util/decimal.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/decimal.cc b/cpp/src/arrow/util/decimal.cc
index 98768cf..bfd87a5 100644
--- a/cpp/src/arrow/util/decimal.cc
+++ b/cpp/src/arrow/util/decimal.cc
@@ -15,19 +15,28 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <algorithm>
 #include <cctype>
-#include <cmath>
+#include <cstdlib>
 #include <sstream>
 
+#include "arrow/util/bit-util.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/int128.h"
+#include "arrow/util/logging.h"
 
 namespace arrow {
 namespace decimal {
 
-template <typename T>
-ARROW_EXPORT Status FromString(const std::string& s, Decimal<T>* out, int* precision,
-                               int* scale) {
+void StringToInteger(const std::string& whole, const std::string& fractional, int8_t sign,
+                     Int128* out) {
+  DCHECK(sign == -1 || sign == 1);
+  DCHECK_NE(out, nullptr);
+  DCHECK(!whole.empty() || !fractional.empty());
+  *out = Int128(whole + fractional) * sign;
+}
+
+Status FromString(const std::string& s, Int128* out, int* precision, int* scale) {
   // Implements this regex: "(\\+?|-?)((0*)(\\d*))(\\.(\\d+))?";
   if (s.empty()) {
     return Status::Invalid("Empty string cannot be converted to decimal");
@@ -56,14 +65,12 @@ ARROW_EXPORT Status FromString(const std::string& s, Decimal<T>* out, int* preci
   DCHECK_LT(charp, end);
 
   // skip leading zeros
-  while (charp != end && *charp == '0') {
-    ++charp;
-  }
+  charp = std::find_if_not(charp, end, [](char c) { return c == '0'; });
 
   // all zeros and no decimal point
   if (charp == end) {
     if (out != nullptr) {
-      out->value = static_cast<T>(0);
+      *out = Int128(0);
     }
 
     // Not sure what other libraries assign precision to for this case (this case of
@@ -81,9 +88,7 @@ ARROW_EXPORT Status FromString(const std::string& s, Decimal<T>* out, int* preci
 
   std::string::const_iterator whole_part_start = charp;
 
-  while (charp != end && isdigit(*charp)) {
-    ++charp;
-  }
+  charp = std::find_if_not(charp, end, [](char c) { return std::isdigit(c) != 0; });
 
   std::string::const_iterator whole_part_end = charp;
   std::string whole_part(whole_part_start, whole_part_end);
@@ -97,7 +102,7 @@ ARROW_EXPORT Status FromString(const std::string& s, Decimal<T>* out, int* preci
           "end of the string.");
     }
 
-    if (!isdigit(*charp)) {
+    if (std::isdigit(*charp) == 0) {
       std::stringstream ss;
       ss << "Decimal point must be followed by a base ten digit. Found '" << *charp
          << "'";
@@ -117,9 +122,7 @@ ARROW_EXPORT Status FromString(const std::string& s, Decimal<T>* out, int* preci
   // The rest must be digits, because if we have a decimal point it must be followed by
   // digits
   if (charp != end) {
-    while (charp != end && isdigit(*charp)) {
-      ++charp;
-    }
+    charp = std::find_if_not(charp, end, [](char c) { return std::isdigit(c) != 0; });
 
     // The while loop has ended before the end of the string which means we've hit a
     // character that isn't a base ten digit
@@ -143,88 +146,64 @@ ARROW_EXPORT Status FromString(const std::string& s, Decimal<T>* out, int* preci
   }
 
   if (out != nullptr) {
-    StringToInteger(whole_part, fractional_part, sign, &out->value);
+    StringToInteger(whole_part, fractional_part, sign, out);
   }
 
   return Status::OK();
 }
 
-template ARROW_EXPORT Status FromString(const std::string& s, Decimal32* out,
-                                        int* precision, int* scale);
-template ARROW_EXPORT Status FromString(const std::string& s, Decimal64* out,
-                                        int* precision, int* scale);
-template ARROW_EXPORT Status FromString(const std::string& s, Decimal128* out,
-                                        int* precision, int* scale);
-
-void StringToInteger(const std::string& whole, const std::string& fractional, int8_t sign,
-                     int32_t* out) {
-  DCHECK(sign == -1 || sign == 1);
-  DCHECK_NE(out, nullptr);
-  DCHECK(!whole.empty() || !fractional.empty());
-
-  if (!whole.empty()) {
-    *out = std::stoi(whole) *
-           static_cast<int32_t>(pow(10.0, static_cast<double>(fractional.size())));
-  }
-  if (!fractional.empty()) {
-    *out += std::stoi(fractional, nullptr, 10);
+std::string ToString(const Int128& decimal_value, int precision, int scale) {
+  Int128 value(decimal_value);
+
+  // Decimal values are sent to clients as strings so in the interest of
+  // speed the string will be created without the using stringstream with the
+  // whole/fractional_part().
+  size_t last_char_idx = precision + (scale > 0)  // Add a space for decimal place
+                         + (scale == precision)   // Add a space for leading 0
+                         + (value < 0);           // Add a space for negative sign
+  std::string str(last_char_idx, '0');
+
+  // Start filling in the values in reverse order by taking the last digit
+  // of the value. Use a positive value and worry about the sign later. At this
+  // point the last_char_idx points to the string terminator.
+  Int128 remaining_value(value);
+
+  size_t first_digit_idx = 0;
+  if (value < 0) {
+    remaining_value = -value;
+    first_digit_idx = 1;
   }
-  *out *= sign;
-}
 
-void StringToInteger(const std::string& whole, const std::string& fractional, int8_t sign,
-                     int64_t* out) {
-  DCHECK(sign == -1 || sign == 1);
-  DCHECK_NE(out, nullptr);
-  DCHECK(!whole.empty() || !fractional.empty());
-  if (!whole.empty()) {
-    *out = static_cast<int64_t>(std::stoll(whole)) *
-           static_cast<int64_t>(pow(10.0, static_cast<double>(fractional.size())));
+  if (scale > 0) {
+    int remaining_scale = scale;
+    do {
+      str[--last_char_idx] =
+          static_cast<char>(remaining_value % 10 + '0');  // Ascii offset
+      remaining_value /= 10;
+    } while (--remaining_scale > 0);
+    str[--last_char_idx] = '.';
+    DCHECK_GT(last_char_idx, first_digit_idx) << "Not enough space remaining";
   }
-  if (!fractional.empty()) {
-    *out += std::stoll(fractional, nullptr, 10);
-  }
-  *out *= sign;
-}
-
-void StringToInteger(const std::string& whole, const std::string& fractional, int8_t sign,
-                     Int128* out) {
-  DCHECK(sign == -1 || sign == 1);
-  DCHECK_NE(out, nullptr);
-  DCHECK(!whole.empty() || !fractional.empty());
-  *out = Int128(whole + fractional) * sign;
-}
-
-void FromBytes(const uint8_t* bytes, Decimal32* decimal) {
-  DCHECK_NE(bytes, nullptr);
-  DCHECK_NE(decimal, nullptr);
-  decimal->value = *reinterpret_cast<const int32_t*>(bytes);
-}
 
-void FromBytes(const uint8_t* bytes, Decimal64* decimal) {
-  DCHECK_NE(bytes, nullptr);
-  DCHECK_NE(decimal, nullptr);
-  decimal->value = *reinterpret_cast<const int64_t*>(bytes);
-}
+  do {
+    str[--last_char_idx] = static_cast<char>(remaining_value % 10 + '0');  // Ascii offset
+    remaining_value /= 10;
+    if (remaining_value == 0) {
+      // Trim any extra leading 0's.
+      if (last_char_idx > first_digit_idx) {
+        str.erase(0, last_char_idx - first_digit_idx);
+      }
 
-void FromBytes(const uint8_t* bytes, Decimal128* decimal) {
-  decimal->value = Int128(bytes);
-}
-
-void ToBytes(const Decimal32& value, uint8_t** bytes) {
-  DCHECK_NE(*bytes, nullptr);
-  *reinterpret_cast<int32_t*>(*bytes) = value.value;
-}
+      break;
+    }
+    // For safety, enforce string length independent of remaining_value.
+  } while (last_char_idx > first_digit_idx);
 
-void ToBytes(const Decimal64& value, uint8_t** bytes) {
-  DCHECK_NE(*bytes, nullptr);
-  *reinterpret_cast<int64_t*>(*bytes) = value.value;
-}
+  if (value < 0) {
+    str[0] = '-';
+  }
 
-void ToBytes(const Decimal128& decimal, uint8_t** bytes) {
-  DCHECK_NE(bytes, nullptr);
-  DCHECK_NE(*bytes, nullptr);
-  decimal.value.ToBytes(bytes);
+  return str;
 }
 
 }  // namespace decimal

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/util/decimal.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/decimal.h b/cpp/src/arrow/util/decimal.h
index 26595ec..4b7cac8 100644
--- a/cpp/src/arrow/util/decimal.h
+++ b/cpp/src/arrow/util/decimal.h
@@ -18,126 +18,22 @@
 #ifndef ARROW_DECIMAL_H
 #define ARROW_DECIMAL_H
 
-#include <cstdint>
-#include <cstdlib>
 #include <string>
 
 #include "arrow/status.h"
-#include "arrow/util/int128.h"  // IWYU pragma: export
-#include "arrow/util/logging.h"
-#include "arrow/util/visibility.h"
 
 namespace arrow {
 namespace decimal {
 
-template <typename T>
-struct ARROW_EXPORT Decimal;
+class Int128;
 
 ARROW_EXPORT void StringToInteger(const std::string& whole, const std::string& fractional,
-                                  int8_t sign, int32_t* out);
-ARROW_EXPORT void StringToInteger(const std::string& whole, const std::string& fractional,
-                                  int8_t sign, int64_t* out);
-ARROW_EXPORT void StringToInteger(const std::string& whole, const std::string& fractional,
                                   int8_t sign, Int128* out);
 
-template <typename T>
-ARROW_EXPORT Status FromString(const std::string& s, Decimal<T>* out,
+ARROW_EXPORT Status FromString(const std::string& string, Int128* out,
                                int* precision = nullptr, int* scale = nullptr);
 
-template <typename T>
-struct ARROW_EXPORT Decimal {
-  Decimal() : value() {}
-  explicit Decimal(const std::string& s) : value() { DCHECK(FromString(s, this).ok()); }
-  explicit Decimal(const char* s) : Decimal(std::string(s)) {}
-  explicit Decimal(const T& value) : value(value) {}
-
-  using value_type = T;
-  value_type value;
-};
-
-using Decimal32 = Decimal<int32_t>;
-using Decimal64 = Decimal<int64_t>;
-using Decimal128 = Decimal<Int128>;
-
-template <typename T>
-struct ARROW_EXPORT DecimalPrecision {};
-
-template <>
-struct ARROW_EXPORT DecimalPrecision<int32_t> {
-  constexpr static const int minimum = 1;
-  constexpr static const int maximum = 9;
-};
-
-template <>
-struct ARROW_EXPORT DecimalPrecision<int64_t> {
-  constexpr static const int minimum = 10;
-  constexpr static const int maximum = 18;
-};
-
-template <>
-struct ARROW_EXPORT DecimalPrecision<Int128> {
-  constexpr static const int minimum = 19;
-  constexpr static const int maximum = 38;
-};
-
-template <typename T>
-ARROW_EXPORT std::string ToString(const Decimal<T>& decimal_value, int precision,
-                                  int scale) {
-  T value = decimal_value.value;
-
-  // Decimal values are sent to clients as strings so in the interest of
-  // speed the string will be created without the using stringstream with the
-  // whole/fractional_part().
-  size_t last_char_idx = precision + (scale > 0)  // Add a space for decimal place
-                         + (scale == precision)   // Add a space for leading 0
-                         + (value < 0);           // Add a space for negative sign
-  std::string str = std::string(last_char_idx, '0');
-  // Start filling in the values in reverse order by taking the last digit
-  // of the value. Use a positive value and worry about the sign later. At this
-  // point the last_char_idx points to the string terminator.
-  T remaining_value = value;
-  size_t first_digit_idx = 0;
-  if (value < 0) {
-    remaining_value = -value;
-    first_digit_idx = 1;
-  }
-  if (scale > 0) {
-    int remaining_scale = scale;
-    do {
-      str[--last_char_idx] =
-          static_cast<char>(remaining_value % 10 + static_cast<T>('0'));  // Ascii offset
-      remaining_value /= 10;
-    } while (--remaining_scale > 0);
-    str[--last_char_idx] = '.';
-    DCHECK_GT(last_char_idx, first_digit_idx) << "Not enough space remaining";
-  }
-  do {
-    str[--last_char_idx] =
-        static_cast<char>(remaining_value % 10 + static_cast<T>('0'));  // Ascii offset
-    remaining_value /= 10;
-    if (remaining_value == 0) {
-      // Trim any extra leading 0's.
-      if (last_char_idx > first_digit_idx) {
-        str.erase(0, last_char_idx - first_digit_idx);
-      }
-
-      break;
-    }
-    // For safety, enforce string length independent of remaining_value.
-  } while (last_char_idx > first_digit_idx);
-  if (value < 0) str[0] = '-';
-  return str;
-}
-
-/// Conversion from raw bytes to a Decimal value
-ARROW_EXPORT void FromBytes(const uint8_t* bytes, Decimal32* value);
-ARROW_EXPORT void FromBytes(const uint8_t* bytes, Decimal64* value);
-ARROW_EXPORT void FromBytes(const uint8_t* bytes, Decimal128* decimal);
-
-/// Conversion from a Decimal value to raw bytes
-ARROW_EXPORT void ToBytes(const Decimal32& value, uint8_t** bytes);
-ARROW_EXPORT void ToBytes(const Decimal64& value, uint8_t** bytes);
-ARROW_EXPORT void ToBytes(const Decimal128& decimal, uint8_t** bytes);
+ARROW_EXPORT std::string ToString(const Int128& decimal_value, int precision, int scale);
 
 }  // namespace decimal
 }  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/util/int128.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/int128.cc b/cpp/src/arrow/util/int128.cc
index 7102097..552c6cc 100644
--- a/cpp/src/arrow/util/int128.cc
+++ b/cpp/src/arrow/util/int128.cc
@@ -1,20 +1,19 @@
-/**
- * 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.
- */
+// 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 <algorithm>
 #include <cmath>
@@ -64,11 +63,15 @@ Int128::Int128(const uint8_t* bytes)
     : Int128(reinterpret_cast<const int64_t*>(bytes)[0],
              reinterpret_cast<const uint64_t*>(bytes)[1]) {}
 
-void Int128::ToBytes(uint8_t** out) const {
-  DCHECK_NE(out, nullptr) << "Cannot fill nullptr of bytes from Int128";
-  DCHECK_NE(*out, nullptr) << "Cannot fill nullptr of bytes from Int128";
+Status Int128::ToBytes(std::array<uint8_t, 16>* out) const {
+  if (out == nullptr) {
+    return Status::Invalid("Cannot fill nullptr of bytes from Int128");
+  }
+
   const uint64_t raw[] = {static_cast<uint64_t>(high_bits_), low_bits_};
-  std::memcpy(*out, raw, 16);
+  const auto* raw_data = reinterpret_cast<const uint8_t*>(raw);
+  std::copy(raw_data, raw_data + out->size(), out->begin());
+  return Status::OK();
 }
 
 Int128& Int128::Negate() {

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/util/int128.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/int128.h b/cpp/src/arrow/util/int128.h
index 19888ae..f6fafd7 100644
--- a/cpp/src/arrow/util/int128.h
+++ b/cpp/src/arrow/util/int128.h
@@ -1,26 +1,27 @@
-/**
- * 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.
- */
+// 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_INT128_H
 #define ARROW_INT128_H
 
+#include <array>
 #include <cstdint>
 #include <string>
+#include <type_traits>
 
 #include "arrow/status.h"
 #include "arrow/util/visibility.h"
@@ -37,18 +38,18 @@ namespace decimal {
 /// Adapted from the Apache ORC C++ implementation
 class ARROW_EXPORT Int128 {
  public:
-  constexpr Int128() : Int128(0, 0) {}
-
-  /// \brief Convert a signed 64 bit value into an Int128.
-  constexpr Int128(int64_t value)
-      : Int128(value >= 0 ? 0 : -1, static_cast<uint64_t>(value)) {}
-
-  /// \brief Convert a signed 32 bit value into an Int128.
-  constexpr Int128(int32_t value) : Int128(static_cast<int64_t>(value)) {}
-
   /// \brief Create an Int128 from the two's complement representation.
   constexpr Int128(int64_t high, uint64_t low) : high_bits_(high), low_bits_(low) {}
 
+  /// \brief Empty constructor creates an Int128 with a value of 0.
+  constexpr Int128() : Int128(0, 0) {}
+
+  /// \brief Convert any integer value into an Int128.
+  template <typename T,
+            typename = typename std::enable_if<std::is_integral<T>::value, T>::type>
+  constexpr Int128(T value)
+      : Int128(static_cast<int64_t>(value) >= 0 ? 0 : -1, static_cast<uint64_t>(value)) {}
+
   /// \brief Parse the number from a base 10 string representation.
   explicit Int128(const std::string& value);
 
@@ -103,7 +104,7 @@ class ARROW_EXPORT Int128 {
   uint64_t low_bits() const { return low_bits_; }
 
   /// \brief Put the raw bytes of the value into a pointer to uint8_t.
-  void ToBytes(uint8_t** out) const;
+  Status ToBytes(std::array<uint8_t, 16>* out) const;
 
  private:
   int64_t high_bits_;

http://git-wip-us.apache.org/repos/asf/arrow/blob/6f2fd8db/cpp/src/arrow/util/macros.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/util/macros.h b/cpp/src/arrow/util/macros.h
index 9352149..ec8d3bd 100644
--- a/cpp/src/arrow/util/macros.h
+++ b/cpp/src/arrow/util/macros.h
@@ -64,7 +64,7 @@
 #if !defined(MANUALLY_ALIGNED_STRUCT)
 #if defined(_MSC_VER)
 #define MANUALLY_ALIGNED_STRUCT(alignment) \
-  __pragma(pack(1));                             \
+  __pragma(pack(1));                       \
   struct __declspec(align(alignment))
 #define STRUCT_END(name, size) \
   __pragma(pack());            \
@@ -77,6 +77,6 @@
 #else
 #error Unknown compiler, please define structure alignment macros
 #endif
-#endif // !defined(MANUALLY_ALIGNED_STRUCT)
+#endif  // !defined(MANUALLY_ALIGNED_STRUCT)
 
 #endif  // ARROW_UTIL_MACROS_H