You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/11/13 04:48:01 UTC

[jira] [Commented] (ARROW-1794) [C++/Python] Rename DecimalArray to Decimal128Array

    [ https://issues.apache.org/jira/browse/ARROW-1794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16249100#comment-16249100 ] 

ASF GitHub Bot commented on ARROW-1794:
---------------------------------------

wesm closed pull request #1312: ARROW-1794: [C++/Python] Rename DecimalArray to Decimal128Array
URL: https://github.com/apache/arrow/pull/1312
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/cpp/src/arrow/array-test.cc b/cpp/src/arrow/array-test.cc
index 9f248cdbb..15c75534e 100644
--- a/cpp/src/arrow/array-test.cc
+++ b/cpp/src/arrow/array-test.cc
@@ -2799,7 +2799,7 @@ class DecimalTest : public ::testing::TestWithParam<int> {
         BitUtil::BytesToBits(valid_bytes, default_memory_pool(), &expected_null_bitmap));
 
     int64_t expected_null_count = test::null_count(valid_bytes);
-    auto expected = std::make_shared<DecimalArray>(
+    auto expected = std::make_shared<Decimal128Array>(
         type, size, expected_data, expected_null_bitmap, expected_null_count);
 
     std::shared_ptr<Array> lhs = out->Slice(offset);
diff --git a/cpp/src/arrow/array.cc b/cpp/src/arrow/array.cc
index 9c91d619c..651fa26ba 100644
--- a/cpp/src/arrow/array.cc
+++ b/cpp/src/arrow/array.cc
@@ -314,12 +314,12 @@ const uint8_t* FixedSizeBinaryArray::GetValue(int64_t i) const {
 // ----------------------------------------------------------------------
 // Decimal
 
-DecimalArray::DecimalArray(const std::shared_ptr<ArrayData>& data)
+Decimal128Array::Decimal128Array(const std::shared_ptr<ArrayData>& data)
     : FixedSizeBinaryArray(data) {
   DCHECK_EQ(data->type->id(), Type::DECIMAL);
 }
 
-std::string DecimalArray::FormatValue(int64_t i) const {
+std::string Decimal128Array::FormatValue(int64_t i) const {
   const auto& type_ = static_cast<const DecimalType&>(*type());
   const Decimal128 value(GetValue(i));
   return value.ToString(type_.scale());
@@ -515,7 +515,7 @@ struct ValidateVisitor {
 
   Status Visit(const PrimitiveArray&) { return Status::OK(); }
 
-  Status Visit(const DecimalArray&) { return Status::OK(); }
+  Status Visit(const Decimal128Array&) { return Status::OK(); }
 
   Status Visit(const BinaryArray&) {
     // TODO(wesm): what to do here?
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index f7762ce10..3337e4b15 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -559,19 +559,22 @@ class ARROW_EXPORT FixedSizeBinaryArray : public PrimitiveArray {
 };
 
 // ----------------------------------------------------------------------
-// DecimalArray
-class ARROW_EXPORT DecimalArray : public FixedSizeBinaryArray {
+// Decimal128Array
+class ARROW_EXPORT Decimal128Array : public FixedSizeBinaryArray {
  public:
   using TypeClass = DecimalType;
 
   using FixedSizeBinaryArray::FixedSizeBinaryArray;
 
-  /// \brief Construct DecimalArray from ArrayData instance
-  explicit DecimalArray(const std::shared_ptr<ArrayData>& data);
+  /// \brief Construct Decimal128Array from ArrayData instance
+  explicit Decimal128Array(const std::shared_ptr<ArrayData>& data);
 
   std::string FormatValue(int64_t i) const;
 };
 
+// Backward compatibility
+using DecimalArray = Decimal128Array;
+
 // ----------------------------------------------------------------------
 // Struct
 
diff --git a/cpp/src/arrow/compare.cc b/cpp/src/arrow/compare.cc
index a2d4de7b7..253c2e1fe 100644
--- a/cpp/src/arrow/compare.cc
+++ b/cpp/src/arrow/compare.cc
@@ -255,7 +255,7 @@ class RangeEqualsVisitor {
     return Status::OK();
   }
 
-  Status Visit(const DecimalArray& left) {
+  Status Visit(const Decimal128Array& left) {
     return Visit(static_cast<const FixedSizeBinaryArray&>(left));
   }
 
diff --git a/cpp/src/arrow/ipc/json-internal.cc b/cpp/src/arrow/ipc/json-internal.cc
index 1b9baee7d..976f9660a 100644
--- a/cpp/src/arrow/ipc/json-internal.cc
+++ b/cpp/src/arrow/ipc/json-internal.cc
@@ -458,7 +458,7 @@ class ArrayWriter {
     }
   }
 
-  void WriteDataValues(const DecimalArray& arr) {
+  void WriteDataValues(const Decimal128Array& arr) {
     for (int64_t i = 0; i < arr.length(); ++i) {
       const Decimal128 value(arr.GetValue(i));
       writer_->String(value.ToIntegerString());
diff --git a/cpp/src/arrow/ipc/test-common.h b/cpp/src/arrow/ipc/test-common.h
index 91023db48..7fc139381 100644
--- a/cpp/src/arrow/ipc/test-common.h
+++ b/cpp/src/arrow/ipc/test-common.h
@@ -690,10 +690,10 @@ Status MakeDecimal(std::shared_ptr<RecordBatch>* out) {
 
   RETURN_NOT_OK(BitUtil::BytesToBits(is_valid_bytes, default_memory_pool(), &is_valid));
 
-  auto a1 = std::make_shared<DecimalArray>(f0->type(), length, data, is_valid,
-                                           kUnknownNullCount);
+  auto a1 = std::make_shared<Decimal128Array>(f0->type(), length, data, is_valid,
+                                              kUnknownNullCount);
 
-  auto a2 = std::make_shared<DecimalArray>(f1->type(), length, data);
+  auto a2 = std::make_shared<Decimal128Array>(f1->type(), length, data);
 
   ArrayVector arrays = {a1, a2};
   *out = std::make_shared<RecordBatch>(schema, length, arrays);
diff --git a/cpp/src/arrow/ipc/writer.cc b/cpp/src/arrow/ipc/writer.cc
index 5598cc682..323116f58 100644
--- a/cpp/src/arrow/ipc/writer.cc
+++ b/cpp/src/arrow/ipc/writer.cc
@@ -339,7 +339,7 @@ class RecordBatchSerializer : public ArrayVisitor {
   VISIT_FIXED_WIDTH(Time32Array)
   VISIT_FIXED_WIDTH(Time64Array)
   VISIT_FIXED_WIDTH(FixedSizeBinaryArray)
-  VISIT_FIXED_WIDTH(DecimalArray)
+  VISIT_FIXED_WIDTH(Decimal128Array)
 
 #undef VISIT_FIXED_WIDTH
 
diff --git a/cpp/src/arrow/pretty_print.cc b/cpp/src/arrow/pretty_print.cc
index aaea34a51..cfbc30315 100644
--- a/cpp/src/arrow/pretty_print.cc
+++ b/cpp/src/arrow/pretty_print.cc
@@ -170,7 +170,7 @@ class ArrayPrinter : public PrettyPrinter {
   }
 
   template <typename T>
-  inline typename std::enable_if<std::is_same<DecimalArray, T>::value, void>::type
+  inline typename std::enable_if<std::is_same<Decimal128Array, T>::value, void>::type
   WriteDataValues(const T& array) {
     for (int i = 0; i < array.length(); ++i) {
       if (i > 0) {
diff --git a/cpp/src/arrow/python/arrow_to_pandas.cc b/cpp/src/arrow/python/arrow_to_pandas.cc
index 3894772da..f966b2987 100644
--- a/cpp/src/arrow/python/arrow_to_pandas.cc
+++ b/cpp/src/arrow/python/arrow_to_pandas.cc
@@ -633,7 +633,7 @@ static Status ConvertDecimals(PandasOptions options, const ChunkedArray& data,
   PyObject* Decimal = Decimal_ref.obj();
 
   for (int c = 0; c < data.num_chunks(); c++) {
-    auto* arr(static_cast<arrow::DecimalArray*>(data.chunk(c).get()));
+    auto* arr(static_cast<arrow::Decimal128Array*>(data.chunk(c).get()));
     auto type(std::dynamic_pointer_cast<arrow::DecimalType>(arr->type()));
     const int scale = type->scale();
 
diff --git a/cpp/src/arrow/type_fwd.h b/cpp/src/arrow/type_fwd.h
index 0d06b6f6c..343487055 100644
--- a/cpp/src/arrow/type_fwd.h
+++ b/cpp/src/arrow/type_fwd.h
@@ -69,7 +69,7 @@ class StructArray;
 class StructBuilder;
 
 class DecimalType;
-class DecimalArray;
+class Decimal128Array;
 class DecimalBuilder;
 
 class UnionType;
diff --git a/cpp/src/arrow/type_traits.h b/cpp/src/arrow/type_traits.h
index fbd78398f..bc600372e 100644
--- a/cpp/src/arrow/type_traits.h
+++ b/cpp/src/arrow/type_traits.h
@@ -231,7 +231,7 @@ struct TypeTraits<DoubleType> {
 
 template <>
 struct TypeTraits<DecimalType> {
-  using ArrayType = DecimalArray;
+  using ArrayType = Decimal128Array;
   using BuilderType = DecimalBuilder;
   constexpr static bool is_parameter_free = false;
 };
diff --git a/cpp/src/arrow/visitor.cc b/cpp/src/arrow/visitor.cc
index a7b01b0f6..3739e89f3 100644
--- a/cpp/src/arrow/visitor.cc
+++ b/cpp/src/arrow/visitor.cc
@@ -56,7 +56,7 @@ ARRAY_VISITOR_DEFAULT(ListArray);
 ARRAY_VISITOR_DEFAULT(StructArray);
 ARRAY_VISITOR_DEFAULT(UnionArray);
 ARRAY_VISITOR_DEFAULT(DictionaryArray);
-ARRAY_VISITOR_DEFAULT(DecimalArray);
+ARRAY_VISITOR_DEFAULT(Decimal128Array);
 
 #undef ARRAY_VISITOR_DEFAULT
 
diff --git a/cpp/src/arrow/visitor.h b/cpp/src/arrow/visitor.h
index 6c36e465e..34679eb95 100644
--- a/cpp/src/arrow/visitor.h
+++ b/cpp/src/arrow/visitor.h
@@ -50,7 +50,7 @@ class ARROW_EXPORT ArrayVisitor {
   virtual Status Visit(const Time64Array& array);
   virtual Status Visit(const TimestampArray& array);
   virtual Status Visit(const IntervalArray& array);
-  virtual Status Visit(const DecimalArray& array);
+  virtual Status Visit(const Decimal128Array& array);
   virtual Status Visit(const ListArray& array);
   virtual Status Visit(const StructArray& array);
   virtual Status Visit(const UnionArray& array);
diff --git a/python/doc/source/api.rst b/python/doc/source/api.rst
index 6bceba3c6..c52024044 100644
--- a/python/doc/source/api.rst
+++ b/python/doc/source/api.rst
@@ -155,7 +155,7 @@ Array Types
    Date32Array
    Date64Array
    TimestampArray
-   DecimalArray
+   Decimal128Array
    ListArray
 
 .. _api.table:
diff --git a/python/pyarrow/__init__.py b/python/pyarrow/__init__.py
index 2d7d7288b..09bf6b35f 100644
--- a/python/pyarrow/__init__.py
+++ b/python/pyarrow/__init__.py
@@ -58,7 +58,7 @@
                          DictionaryArray,
                          Date32Array, Date64Array,
                          TimestampArray, Time32Array, Time64Array,
-                         DecimalArray, StructArray,
+                         Decimal128Array, StructArray,
                          ArrayValue, Scalar, NA,
                          BooleanValue,
                          Int8Value, Int16Value, Int32Value, Int64Value,
diff --git a/python/pyarrow/array.pxi b/python/pyarrow/array.pxi
index 9991411e5..2ef592ff7 100644
--- a/python/pyarrow/array.pxi
+++ b/python/pyarrow/array.pxi
@@ -597,7 +597,7 @@ cdef class FixedSizeBinaryArray(Array):
     pass
 
 
-cdef class DecimalArray(FixedSizeBinaryArray):
+cdef class Decimal128Array(FixedSizeBinaryArray):
     pass
 
 
@@ -846,7 +846,7 @@ cdef dict _array_classes = {
     _Type_STRING: StringArray,
     _Type_DICTIONARY: DictionaryArray,
     _Type_FIXED_SIZE_BINARY: FixedSizeBinaryArray,
-    _Type_DECIMAL: DecimalArray,
+    _Type_DECIMAL: Decimal128Array,
     _Type_STRUCT: StructArray,
 }
 
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index dfafd371b..11cc6b3ff 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -312,7 +312,9 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
     cdef cppclass CFixedSizeBinaryArray" arrow::FixedSizeBinaryArray"(CArray):
         const uint8_t* GetValue(int i)
 
-    cdef cppclass CDecimalArray" arrow::DecimalArray"(CFixedSizeBinaryArray):
+    cdef cppclass CDecimal128Array" arrow::Decimal128Array"(
+        CFixedSizeBinaryArray
+    ):
         c_string FormatValue(int i)
 
     cdef cppclass CListArray" arrow::ListArray"(CArray):
diff --git a/python/pyarrow/lib.pxd b/python/pyarrow/lib.pxd
index 531489490..6413b838f 100644
--- a/python/pyarrow/lib.pxd
+++ b/python/pyarrow/lib.pxd
@@ -246,7 +246,7 @@ cdef class FixedSizeBinaryArray(Array):
     pass
 
 
-cdef class DecimalArray(FixedSizeBinaryArray):
+cdef class Decimal128Array(FixedSizeBinaryArray):
     pass
 
 
diff --git a/python/pyarrow/scalar.pxi b/python/pyarrow/scalar.pxi
index a396fa763..1bc5ed7a3 100644
--- a/python/pyarrow/scalar.pxi
+++ b/python/pyarrow/scalar.pxi
@@ -258,7 +258,7 @@ cdef class DecimalValue(ArrayValue):
 
     def as_py(self):
         cdef:
-            CDecimalArray* ap = <CDecimalArray*> self.sp_array.get()
+            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
             c_string s = ap.FormatValue(self.index)
         return _pydecimal.Decimal(s.decode('utf8'))
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> [C++/Python] Rename DecimalArray to Decimal128Array
> ---------------------------------------------------
>
>                 Key: ARROW-1794
>                 URL: https://issues.apache.org/jira/browse/ARROW-1794
>             Project: Apache Arrow
>          Issue Type: Improvement
>          Components: C++
>    Affects Versions: 0.7.1
>            Reporter: Wes McKinney
>            Assignee: Phillip Cloud
>              Labels: pull-request-available
>             Fix For: 0.8.0
>
>
> This will help future proof the API for when we may need to add smaller in-memory decimal types



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)