You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/02/08 13:17:27 UTC

[GitHub] [arrow] lidavidm commented on a change in pull request #12315: ARROW-15389: [C++][Dev] Improve Array preview in GDB plugin

lidavidm commented on a change in pull request #12315:
URL: https://github.com/apache/arrow/pull/12315#discussion_r801611235



##########
File path: python/pyarrow/tests/test_gdb.py
##########
@@ -772,27 +784,227 @@ def test_scalars_heap(gdb_arrow):
 def test_array_data(gdb_arrow):
     check_stack_repr(
         gdb_arrow, "int32_array_data",
-        "arrow::ArrayData of type arrow::int32(), length 4, null count 1")
+        ("arrow::ArrayData of type arrow::int32(), length 4, offset 0, "
+         "null count 1 = {[0] = -5, [1] = 6, [2] = null, [3] = 42}"))
 
 
 def test_arrays_stack(gdb_arrow):
     check_stack_repr(
         gdb_arrow, "int32_array",
-        "arrow::Int32Array of length 4, null count 1")
+        ("arrow::Int32Array of length 4, offset 0, null count 1 = "
+         "{[0] = -5, [1] = 6, [2] = null, [3] = 42}"))
     check_stack_repr(
         gdb_arrow, "list_array",
         ("arrow::ListArray of type arrow::list(arrow::int64()), "
-         "length 3, null count 1"))
+         "length 3, offset 0, null count 1"))
 
 
 def test_arrays_heap(gdb_arrow):
+    # Null
+    check_heap_repr(
+        gdb_arrow, "heap_null_array",
+        "arrow::NullArray of length 2, offset 0, null count 2")
+
+    # Primitive
     check_heap_repr(
         gdb_arrow, "heap_int32_array",
-        "arrow::Int32Array of length 4, null count 1")
+        ("arrow::Int32Array of length 4, offset 0, null count 1 = {"
+         "[0] = -5, [1] = 6, [2] = null, [3] = 42}"))
+    check_heap_repr(
+        gdb_arrow, "heap_int32_array_no_nulls",
+        ("arrow::Int32Array of length 4, offset 0, null count 0 = {"
+         "[0] = -5, [1] = 6, [2] = 3, [3] = 42}"))
+    check_heap_repr(
+        gdb_arrow, "heap_int32_array_sliced_1_9",
+        ("arrow::Int32Array of length 9, offset 1, unknown null count = {"
+         "[0] = 2, [1] = -3, [2] = 4, [3] = null, [4] = -5, [5] = 6, "
+         "[6] = -7, [7] = 8, [8] = null}"))
+    check_heap_repr(
+        gdb_arrow, "heap_int32_array_sliced_2_6",
+        ("arrow::Int32Array of length 6, offset 2, unknown null count = {"
+         "[0] = -3, [1] = 4, [2] = null, [3] = -5, [4] = 6, [5] = -7}"))
+    check_heap_repr(
+        gdb_arrow, "heap_int32_array_sliced_8_4",
+        ("arrow::Int32Array of length 4, offset 8, unknown null count = {"
+         "[0] = 8, [1] = null, [2] = -9, [3] = -10}"))
+    check_heap_repr(
+        gdb_arrow, "heap_int32_array_sliced_empty",
+        "arrow::Int32Array of length 0, offset 6, unknown null count")
+
+    check_heap_repr(
+        gdb_arrow, "heap_double_array",
+        ("arrow::DoubleArray of length 2, offset 0, null count 1 = {"
+         "[0] = -1.5, [1] = null}"))
+    check_heap_repr(
+        gdb_arrow, "heap_float16_array",
+        ("arrow::HalfFloatArray of length 2, offset 0, null count 0 = {"
+         "[0] = 0.0, [1] = -1.5}"))
+
+    # Boolean
+    check_heap_repr(
+        gdb_arrow, "heap_bool_array",
+        ("arrow::BooleanArray of length 18, offset 0, null count 6 = {"
+         "[0] = false, [1] = false, [2] = true, [3] = true, [4] = null, "
+         "[5] = null, [6] = false, [7] = false, [8] = true, [9] = true, "
+         "[10] = null, [11] = null, [12] = false, [13] = false, "
+         "[14] = true, [15] = true, [16] = null, [17] = null}"))

Review comment:
       Do we want to cut off or ellipsize the repr if the array is too long? Otherwise it might be surprising during a debugging session.

##########
File path: cpp/src/arrow/python/gdb.cc
##########
@@ -394,15 +408,100 @@ void TestSession() {
   auto heap_map_scalar_null = MakeNullScalar(heap_map_scalar->type);
 
   // Array and ArrayData
-  std::shared_ptr<Array> heap_int32_array;
-  ARROW_CHECK_OK(ArrayFromJSON(int32(), "[-5, 6, null, 42]", &heap_int32_array));
+  auto heap_null_array = SliceArrayFromJSON(null(), "[null, null]");
+
+  auto heap_int32_array = SliceArrayFromJSON(int32(), "[-5, 6, null, 42]");
   ArrayData int32_array_data{*heap_int32_array->data()};
   Int32Array int32_array{heap_int32_array->data()->Copy()};
 
-  std::shared_ptr<Array> heap_list_array;
-  ARROW_CHECK_OK(ArrayFromJSON(list(int64()), "[[1, 2], null, []]", &heap_list_array));
+  auto heap_int32_array_no_nulls = SliceArrayFromJSON(int32(), "[-5, 6, 3, 42]");
+
+  const char* json_int32_array = "[-1, 2, -3, 4, null, -5, 6, -7, 8, null, -9, -10]";
+  auto heap_int32_array_sliced_1_9 = SliceArrayFromJSON(int32(), json_int32_array, 1, 9);
+  auto heap_int32_array_sliced_2_6 = SliceArrayFromJSON(int32(), json_int32_array, 2, 6);
+  auto heap_int32_array_sliced_8_4 = SliceArrayFromJSON(int32(), json_int32_array, 8, 4);
+  auto heap_int32_array_sliced_empty =
+      SliceArrayFromJSON(int32(), json_int32_array, 6, 0);
+
+  const char* json_bool_array =
+      "[false, false, true, true, null, null, false, false, true, true, "
+      "null, null, false, false, true, true, null, null]";
+  auto heap_bool_array = SliceArrayFromJSON(boolean(), json_bool_array);
+  auto heap_bool_array_sliced_1_9 = SliceArrayFromJSON(boolean(), json_bool_array, 1, 9);
+  auto heap_bool_array_sliced_2_6 = SliceArrayFromJSON(boolean(), json_bool_array, 2, 6);
+  auto heap_bool_array_sliced_empty =
+      SliceArrayFromJSON(boolean(), json_bool_array, 6, 0);
+
+  auto heap_list_array = SliceArrayFromJSON(list(int64()), "[[1, 2], null, []]");
   ListArray list_array{heap_list_array->data()};
 
+  const char* json_double_array = "[-1.5, null]";
+  auto heap_double_array = SliceArrayFromJSON(float64(), json_double_array);
+
+  const char* json_float16_array = "[0, 48640]";
+  auto heap_float16_array =
+      *SliceArrayFromJSON(uint16(), json_float16_array)->View(float16());
+
+  auto heap_date32_array =
+      SliceArrayFromJSON(date32(), "[0, null, 18336, -9004, -719162, -719163]");
+  auto heap_date64_array = SliceArrayFromJSON(
+      date64(), "[1584230400000, -777945600000, -62135596800000, -62135683200000, 123]");
+
+  const char* json_time_array = "[null, -123, 456]";
+  auto heap_time32_array_s =
+      SliceArrayFromJSON(time32(TimeUnit::SECOND), json_time_array);
+  auto heap_time32_array_ms =
+      SliceArrayFromJSON(time32(TimeUnit::MILLI), json_time_array);
+  auto heap_time64_array_us =
+      SliceArrayFromJSON(time64(TimeUnit::MICRO), json_time_array);
+  auto heap_time64_array_ns = SliceArrayFromJSON(time64(TimeUnit::NANO), json_time_array);
+
+  auto heap_month_interval_array =
+      SliceArrayFromJSON(month_interval(), "[123, -456, null]");
+  auto heap_day_time_interval_array =
+      SliceArrayFromJSON(day_time_interval(), "[[1, -600], null]");
+  auto heap_month_day_nano_interval_array =
+      SliceArrayFromJSON(month_day_nano_interval(), "[[1, -600, 5000], null]");
+
+  const char* json_duration_array = "[null, -1234567890123456789]";
+  auto heap_duration_array_s =
+      SliceArrayFromJSON(duration(TimeUnit::SECOND), json_duration_array);
+  auto heap_duration_array_ns =
+      SliceArrayFromJSON(duration(TimeUnit::NANO), json_duration_array);
+
+  auto heap_timestamp_array_s = SliceArrayFromJSON(
+      timestamp(TimeUnit::SECOND),
+      R"([null, "1970-01-01 00:00:00", "1900-02-28 12:34:56", "3989-07-14 00:00:00"])");
+  auto heap_timestamp_array_ms = SliceArrayFromJSON(
+      timestamp(TimeUnit::MILLI),
+      R"([null, "1900-02-28 12:34:56.123", "3989-07-14 00:00:00.789"])");
+  auto heap_timestamp_array_us = SliceArrayFromJSON(
+      timestamp(TimeUnit::MICRO),
+      R"([null, "1900-02-28 12:34:56.654321", "3989-07-14 00:00:00.456789"])");
+  auto heap_timestamp_array_ns = SliceArrayFromJSON(
+      timestamp(TimeUnit::NANO), R"([null, "1900-02-28 12:34:56.987654321"])");
+
+  auto heap_decimal128_array = SliceArrayFromJSON(
+      decimal128(30, 6),
+      R"([null, "-1234567890123456789.012345", "1234567890123456789.012345"])");
+  auto heap_decimal256_array = SliceArrayFromJSON(
+      decimal256(50, 6), R"([null, "-123456789012345678901234567890123456789.012345"])");
+  auto heap_decimal128_array_sliced = heap_decimal128_array->Slice(1, 1);
+
+  auto heap_fixed_size_binary_array =
+      SliceArrayFromJSON(fixed_size_binary(3), "[null, \"abc\", \"\\u0000\\u001f\xff\"]");
+  auto heap_fixed_size_binary_array_zero_width =
+      SliceArrayFromJSON(fixed_size_binary(0), R"([null, ""])");
+  auto heap_fixed_size_binary_array_sliced = heap_fixed_size_binary_array->Slice(1, 1);
+
+  const char* json_binary_array = "[null, \"abcd\", \"\\u0000\\u001f\xff\"]";
+  auto heap_binary_array = SliceArrayFromJSON(binary(), json_binary_array);
+  auto heap_large_binary_array = SliceArrayFromJSON(large_binary(), json_binary_array);
+  const char* json_string_array = "[null, \"héhé\", \"invalid \xff char\"]";
+  auto heap_string_array = SliceArrayFromJSON(utf8(), json_string_array);
+  auto heap_large_string_array = SliceArrayFromJSON(large_utf8(), json_string_array);
+  auto heap_binary_array_sliced = heap_binary_array->Slice(1, 1);
+

Review comment:
       How do union arrays print?

##########
File path: cpp/gdb_arrow.py
##########
@@ -1230,15 +1532,245 @@ def type(self):
         return cast_to_concrete(deref(self.val['type']), concrete_type)
 
     def _format_contents(self):
-        return (f"length {self.val['length']}, "
+        return (f"length {self.length}, "
+                f"offset {self.offset}, "
                 f"{format_null_count(self.val['null_count'])}")
 
+    def _buffer(self, index, type_id=None):
+        buffers = StdVector(self.val['buffers'])
+        bufptr = SharedPtr(buffers[index]).get()
+        if int(bufptr) == 0:
+            return None
+        if type_id is not None:
+            return TypedBuffer.from_type_id(bufptr.dereference(), type_id)
+        else:
+            return Buffer(bufptr.dereference())
+
+    def _buffer_values(self, index, type_id, length=None):
+        """
+        Return a typed view of values in the buffer with the given index.
+
+        Values are returned as tuples since some types may decode to
+        multiple values (for example day_time_interval).
+        """
+        buf = self._buffer(index, type_id)
+        if buf is None:
+            return None
+        if length is None:
+            length = self.length
+        return buf.view(self.offset, length)
+
+    def _unpacked_buffer_values(self, index, type_id, length=None):
+        """
+        Like _buffer_values(), but assumes values are 1-tuples
+        and returns them unpacked.
+        """
+        return StarMappedView(identity,
+                              self._buffer_values(index, type_id, length))
+
+    def _null_bitmap(self):
+        buf = self._buffer(0) if has_null_bitmap(self.type_id) else None
+        return NullBitmap.from_buffer(buf, self.offset, self.length)
+
+    def display_hint(self):
+        return None
+
+    def children(self):
+        return ()
+
     def to_string(self):
         ty = self.type
         return (f"{self.name} of type {ty}, "
                 f"{self._format_contents()}")
 
 
+class NumericArrayDataPrinter(ArrayDataPrinter):
+    """
+    ArrayDataPrinter specialization for numeric data types.
+    """
+    _format_value = staticmethod(identity)
+
+    def _values_view(self):
+        return StarMappedView(self._format_value,
+                              self._buffer_values(1, self.type_id))
+
+    def display_hint(self):
+        return "array"
+
+    def children(self):

Review comment:
       Would it make sense to push this loop for printing values up into the superclass or into ArrayPrinter, and then subclasses only have to worry about formatting individual values? It would presumably then be easier to cut off or omit values to shorten the representation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org