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 2020/07/01 18:43:16 UTC

[GitHub] [arrow] kszucs commented on a change in pull request #7519: ARROW-9017: [C++][Python] Refactor scalar bindings

kszucs commented on a change in pull request #7519:
URL: https://github.com/apache/arrow/pull/7519#discussion_r448550619



##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,748 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
-    """
-
-
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
+    The base class for scalars.
     """
-    # TODO rename this NullValue?
-
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
-
-        self.type = null()
-
-    def __repr__(self):
-        return 'NULL'
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-    def __eq__(self, other):
-        return NA
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
 
-_NULL = NA = NullType()
+        if type_id == _Type_NA:
+            return _NULL
 
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
 
-cdef class ArrayValue(Scalar):
-    """
-    The base class for non-null array elements.
-    """
+        return self
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
 
-    cdef void init(self, DataType type, const shared_ptr[CArray]& sp_array,
-                   int64_t index):
-        self.type = type
-        self.index = index
-        self._set_array(sp_array)
+    @property
+    def type(self):
+        """
+        Data type of the Scalar object.
+        """
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    @property
+    def is_valid(self):
+        """
+        Holds a valid (non-null) value.
+        """
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
     def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        if hasattr(self, 'as_py'):
-            if isinstance(other, ArrayValue):
-                other = other.as_py()
-            return self.as_py() == other
-        else:
-            raise NotImplementedError(
-                "Cannot compare Arrow values that don't support as_py()")
+        try:
+            if not isinstance(other, Scalar):
+                other = scalar(other, type=self.type)
+            return self.equals(other)
+        except (TypeError, ValueError, ArrowInvalid):
+            return NotImplemented
 
     def __hash__(self):
-        return hash(self.as_py())
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
+    def as_py(self):
+        raise NotImplementedError()
 
-cdef class BooleanValue(ArrayValue):
-    """
-    Concrete class for boolean array elements.
-    """
 
-    def as_py(self):
-        """
-        Return this value as a Python bool.
-        """
-        cdef CBooleanArray* ap = <CBooleanArray*> self.sp_array.get()
-        return ap.Value(self.index)
+_NULL = NA = None
 
 
-cdef class Int8Value(ArrayValue):
+cdef class NullScalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for null scalars.
     """
 
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt8Array* ap = <CInt8Array*> self.sp_array.get()
-        return ap.Value(self.index)
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise RuntimeError('Cannot create multiple NullScalar instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
+    def __init__(self):
+        pass
 
-cdef class UInt8Value(ArrayValue):
-    """
-    Concrete class for uint8 array elements.
-    """
+    def __eq__(self, other):
+        return NA
+
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this value as a Python None.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> self.sp_array.get()
-        return ap.Value(self.index)
-
+        return None
 
-cdef class Int16Value(ArrayValue):
-    """
-    Concrete class for int16 array elements.
-    """
 
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt16Array* ap = <CInt16Array*> self.sp_array.get()
-        return ap.Value(self.index)
+_NULL = NA = NullScalar()
 
 
-cdef class UInt16Value(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this value as a Python bool.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CBooleanScalar* sp = <CBooleanScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int32Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt32Array* ap = <CInt32Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt32Array* ap = <CUInt32Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CInt8Scalar* sp = <CInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt64Array* ap = <CInt64Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CUInt16Scalar* sp = <CUInt16Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt64Array* ap = <CUInt64Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CInt16Scalar* sp = <CInt16Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for uint32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python int.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
-
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+        cdef CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for int32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python int.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for uint64 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python int.
         """
-        cdef:
-            CTime32Array* ap = <CTime32Array*> self.sp_array.get()
-            CTime32Type* dtype = <CTime32Type*> ap.type().get()
-
-        if dtype.unit() == TimeUnit_SECOND:
-            delta = datetime.timedelta(seconds=ap.Value(self.index))
-            return (datetime.datetime(1970, 1, 1) + delta).time()
-        else:
-            return _box_time_milli(ap.Value(self.index))
+        cdef CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for int64 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python int.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
-
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
-        else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
-
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
-
-
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
-
-
-cdef dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
-
-
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
-    if tzinfo is not None:
-        if not isinstance(tzinfo, datetime.tzinfo):
-            tzinfo = string_to_tzinfo(tzinfo)
-        dt = tzinfo.fromutc(dt)
-    return dt
-
-
-cdef _datetime_from_seconds(int64_t v):
-    return datetime.datetime(1970, 1, 1) + datetime.timedelta(seconds=v)
-
-
-def _nanoseconds_to_datetime_safe(v, tzinfo):
-    if v % 1000 != 0:
-        raise ValueError("Nanosecond timestamp {} is not safely convertible "
-                         " to microseconds to convert to datetime.datetime."
-                         " Install pandas to return as Timestamp with "
-                         " nanosecond support or access the .value attribute.")
-    v = v // 1000
-    micros = v % 1_000_000
-
-    dt = _datetime_from_seconds(v // 1_000_000)
-    return _add_micros_maybe_localize(dt, micros, tzinfo)
-
-
-def _microseconds_to_datetime(v, tzinfo):
-    micros = v % 1_000_000
-    dt = _datetime_from_seconds(v // 1_000_000)
-    return _add_micros_maybe_localize(dt, micros, tzinfo)
-
-
-def _millis_to_datetime(v, tzinfo):
-    millis = v % 1_000
-    dt = _datetime_from_seconds(v // 1000)
-    return _add_micros_maybe_localize(dt, millis * 1000, tzinfo)
-
-
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
-
-
-def _datetime_conversion_functions():
-    global _datetime_conversion_initialized
-    if _datetime_conversion_initialized:
-        return _DATETIME_CONVERSION_FUNCTIONS
-
-    _DATETIME_CONVERSION_FUNCTIONS.update({
-        TimeUnit_SECOND: _seconds_to_datetime,
-        TimeUnit_MILLI: _millis_to_datetime,
-        TimeUnit_MICRO: _microseconds_to_datetime,
-        TimeUnit_NANO: _nanoseconds_to_datetime_safe
-    })
-
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
-
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        cdef CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for float scalars.
     """
 
-    @property
-    def value(self):
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
-        return ap.Value(self.index)
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if isinstance(other, Scalar):
+            other = other.as_py()
+        return self.as_py() == other
 
     def as_py(self):
         """
-        Return this value as a Pandas Timestamp instance (if available),
-        otherwise as a Python datetime.timedelta instance.
+        Return this value as a Python float.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
-
-        value = self.value
-
-        if not dtype.timezone().empty():
-            tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
-        else:
-            tzinfo = None
-
-        try:
-            converter = _datetime_conversion_functions()[dtype.unit()]
-        except KeyError:
-            raise ValueError(
-                'Cannot convert nanosecond timestamps without pandas'
-            )
-        return converter(value, tzinfo=tzinfo)
-
-
-cdef dict _TIMEDELTA_CONVERSION_FUNCTIONS = {}
-
-
-def _nanoseconds_to_timedelta_safe(v):
-    if v % 1000 != 0:
-        raise ValueError(
-            "Nanosecond duration {} is not safely convertible to microseconds "
-            "to convert to datetime.timedelta. Install pandas to return as "
-            "Timedelta with nanosecond support or access the .value "
-            "attribute.".format(v))
-    micros = v // 1000
-
-    return datetime.timedelta(microseconds=micros)
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
 
-def _timedelta_conversion_functions():
-    if _TIMEDELTA_CONVERSION_FUNCTIONS:
-        return _TIMEDELTA_CONVERSION_FUNCTIONS
-
-    _TIMEDELTA_CONVERSION_FUNCTIONS.update({
-        TimeUnit_SECOND: lambda v: datetime.timedelta(seconds=v),
-        TimeUnit_MILLI: lambda v: datetime.timedelta(milliseconds=v),
-        TimeUnit_MICRO: lambda v: datetime.timedelta(microseconds=v),
-        TimeUnit_NANO: _nanoseconds_to_timedelta_safe
-    })
-
-    try:
-        import pandas as pd
-        _TIMEDELTA_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda v: pd.Timedelta(v, unit='ns')
-        )
-    except ImportError:
-        pass
-
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
-
-
-cdef class DurationValue(ArrayValue):
+cdef class FloatScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for float scalars.
     """
 
-    @property
-    def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
-
     def as_py(self):
         """
-        Return this value as a Pandas Timestamp instance (if available),
-        otherwise as a Python datetime.timedelta instance.
+        Return this value as a Python float.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
-
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class HalfFloatValue(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python float.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class FloatValue(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for float32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python Decimal.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef:
+            CDecimal128Scalar* sp = <CDecimal128Scalar*> self.wrapped.get()
+            CDecimal128Type* dtype = <CDecimal128Type*> sp.type.get()
+        if sp.is_valid:
+            return _pydecimal.Decimal(
+                frombytes(sp.value.ToString(dtype.scale()))
+            )
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for float64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
+
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
+        else:
+            return None
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class Date64Scalar(Scalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for date64 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        cdef CDate64Scalar* sp = <CDate64Scalar*> self.wrapped.get()
 
+        if sp.is_valid:
+            return (
+                datetime.date(1970, 1, 1) +
+                datetime.timedelta(days=sp.value / 86400000)
+            )
+        else:
+            return None
 
-cdef class StringValue(ArrayValue):
-    """
-    Concrete class for string (utf8) array elements.
-    """
 
-    def as_py(self):
-        """
-        Return this value as a Python unicode string.
-        """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+def _datetime_from_int(int64_t value, TimeUnit unit, tzinfo=None):
+    if unit == TimeUnit_SECOND:
+        delta = datetime.timedelta(seconds=value)
+    elif unit == TimeUnit_MILLI:
+        delta = datetime.timedelta(milliseconds=value)
+    elif unit == TimeUnit_MICRO:
+        delta = datetime.timedelta(microseconds=value)
+    else:
+        # TimeUnit_NANO: prefer pandas timestamps if available
+        if _pandas_api.have_pandas:
+            return _pandas_api.pd.Timestamp(value, tz=tzinfo, unit='ns')
+        # otherwise safely truncate to microsecond resolution datetime
+        if value % 1000 != 0:
+            raise ValueError(
+                "Nanosecond resolution temporal type {} is not safely "
+                "convertible to microseconds to convert to datetime.datetime. "
+                "Install pandas to return as Timestamp with nanosecond "
+                "support or access the .value attribute.".format(value)
+            )
+        delta = datetime.timedelta(microseconds=value // 1000)
 
-    def as_buffer(self):
-        """
-        Return a view over this value as a Buffer object.
-        """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
+    if tzinfo is not None:
+        dt = tzinfo.fromutc(dt)
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+    return dt
 
 
-cdef class LargeStringValue(ArrayValue):
+cdef class Time32Scalar(Scalar):
     """
-    Concrete class for large string (utf8) array elements.
+    Concrete class for time32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python unicode string.
-        """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
-
-    def as_buffer(self):
-        """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python datetime.timedelta instance.
         """
         cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class Time64Scalar(Scalar):
     """
-    Concrete class for variable-sized binary array elements.
+    Concrete class for time64 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            const uint8_t* ptr
-            int32_t length
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
-
-    def as_buffer(self):
-        """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python datetime.timedelta instance.
         """
         cdef:
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class LargeBinaryValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for large variable-sized binary array elements.
+    Concrete class for timestamp scalars.
     """
 
+    @property
+    def value(self):
+        cdef CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
+
     def as_py(self):
         """
-        Return this value as a Python bytes object.
+        Return this value as a Pandas Timestamp instance (if available),
+        otherwise as a Python datetime.timedelta instance.
         """
         cdef:
-            const uint8_t* ptr
-            int64_t length
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
+        if not sp.is_valid:
+            return None
 
-    def as_buffer(self):
-        """
-        Return a view over this value as a Buffer object.
-        """
-        cdef:
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        if not dtype.timezone().empty():
+            tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+        else:
+            tzinfo = None
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        return _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
 
-cdef class ListValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for list array elements.
+    Concrete class for duration scalars.
     """
 
-    def __len__(self):
-        """
-        Return the number of values.
-        """
-        return self.length()
-
-    def __getitem__(self, i):
-        """
-        Return the value at the given index.
-        """
-        return self.getitem(_normalize_index(i, self.length()))
-
-    def __iter__(self):
-        """
-        Iterate over this element's values.
-        """
-        for i in range(len(self)):
-            yield self.getitem(i)
-        raise StopIteration
-
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CListArray*> sp_array.get()
-        self.value_type = pyarrow_wrap_data_type(self.ap.value_type())
-
-    cdef getitem(self, int64_t i):
-        cdef int64_t j = self.ap.value_offset(self.index) + i
-        return box_scalar(self.value_type, self.ap.values(), j)
-
-    cdef int64_t length(self):
-        return self.ap.value_length(self.index)
+    @property
+    def value(self):
+        cdef CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python list.
+        Return this value as a Pandas Timestamp instance (if available),
+        otherwise as a Python datetime.timedelta instance.
         """
         cdef:
-            int64_t j
-            list result = []
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
 
-        for j in range(len(self)):
-            result.append(self.getitem(j).as_py())
+        if not sp.is_valid:
+            return None
 
-        return result
+        if unit == TimeUnit_SECOND:
+            return datetime.timedelta(seconds=sp.value)
+        elif unit == TimeUnit_MILLI:
+            return datetime.timedelta(milliseconds=sp.value)
+        elif unit == TimeUnit_MICRO:
+            return datetime.timedelta(microseconds=sp.value)
+        else:
+            # TimeUnit_NANO: prefer pandas timestamps if available
+            if _pandas_api.have_pandas:
+                return _pandas_api.pd.Timedelta(sp.value, unit='ns')
+            # otherwise safely truncate to microsecond resolution timedelta
+            if sp.value % 1000 != 0:
+                raise ValueError(
+                    "Nanosecond duration {} is not safely convertible to "
+                    "microseconds to convert to datetime.timedelta. Install "
+                    "pandas to return as Timedelta with nanosecond support or "
+                    "access the .value attribute.".format(sp.value)
+                )
+            return datetime.timedelta(microseconds=sp.value // 1000)
 
 
-cdef class LargeListValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for large list array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def __len__(self):
-        """
-        Return the number of values.
-        """
-        return self.length()
-
-    def __getitem__(self, i):
+    def as_buffer(self):
         """
-        Return the value at the given index.
+        Return a view over this value as a Buffer object.
         """
-        return self.getitem(_normalize_index(i, self.length()))
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
-    def __iter__(self):
+    def as_py(self):
         """
-        Iterate over this element's values.
+        Return this value as a Python bytes.
         """
-        for i in range(len(self)):
-            yield self.getitem(i)
-        raise StopIteration
+        buffer = self.as_buffer()
+        return None if buffer is None else buffer.to_pybytes()
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CLargeListArray*> sp_array.get()
-        self.value_type = pyarrow_wrap_data_type(self.ap.value_type())
 
-    cdef getitem(self, int64_t i):
-        cdef int64_t j = self.ap.value_offset(self.index) + i
-        return box_scalar(self.value_type, self.ap.values(), j)
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    cdef int64_t length(self):
-        return self.ap.value_length(self.index)
 
-    def as_py(self):
-        """
-        Return this value as a Python list.
-        """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            result.append(self.getitem(j).as_py())
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
-        return result
 
-
-cdef class MapValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for map array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
-    def __len__(self):
-        """
-        Return the number of values.
-        """
-        return self.length()
-
-    def __getitem__(self, i):
-        """
-        Return the value at the given index.
-        """
-        return self.getitem(_normalize_index(i, self.length()))
-
-    def __iter__(self):
-        """
-        Iterate over this element's values.
-        """
-        for i in range(len(self)):
-            yield self.getitem(i)
-        raise StopIteration
-
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CMapArray*> sp_array.get()
-        self.key_type = pyarrow_wrap_data_type(self.ap.map_type().key_type())
-        self.item_type = pyarrow_wrap_data_type(self.ap.map_type().item_type())
-
-    cdef getitem(self, int64_t i):
-        cdef int64_t j = self.ap.value_offset(self.index) + i
-        return (box_scalar(self.key_type, self.ap.keys(), j),
-                box_scalar(self.item_type, self.ap.items(), j))
-
-    cdef int64_t length(self):
-        return self.ap.value_length(self.index)
-
     def as_py(self):
         """
-        Return this value as a Python list of tuples, each containing a
-        key and item.
+        Return this value as a Python string.
         """
-        cdef:
-            int64_t j
-            list result = []
+        buffer = self.as_buffer()
+        return None if buffer is None else str(buffer, 'utf8')
 
-        for j in range(len(self)):
-            key, item = self.getitem(j)
-            result.append((key.as_py(), item.as_py()))
 
-        return result
+cdef class LargeStringScalar(StringScalar):
+    pass
 
 
-cdef class FixedSizeListValue(ArrayValue):
+cdef class ListScalar(Scalar):
     """
-    Concrete class for fixed size list array elements.
+    Concrete class for list-like scalars.
     """
 
+    @property
+    def values(self):
+        cdef CListScalar* sp = <CListScalar*> self.wrapped.get()
+        if sp.is_valid:
+            return pyarrow_wrap_array(sp.value)
+        else:
+            return None
+
     def __len__(self):
         """
         Return the number of values.
         """
-        return self.length()
+        return len(self.values)
 
     def __getitem__(self, i):
         """
         Return the value at the given index.
         """
-        return self.getitem(_normalize_index(i, self.length()))
+        return self.values[_normalize_index(i, len(self))]
 
     def __iter__(self):
         """
         Iterate over this element's values.
         """
-        for i in range(len(self)):
-            yield self.getitem(i)
-        raise StopIteration
-
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CFixedSizeListArray*> sp_array.get()
-        self.value_type = pyarrow_wrap_data_type(self.ap.value_type())
-
-    cdef getitem(self, int64_t i):
-        cdef int64_t j = self.ap.value_offset(self.index) + i
-        return box_scalar(self.value_type, self.ap.values(), j)
-
-    cdef int64_t length(self):
-        return self.ap.value_length(self.index)
+        return iter(self.values)
 
     def as_py(self):
         """
         Return this value as a Python list.
         """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            result.append(self.getitem(j).as_py())
-
-        return result
-
-
-cdef class UnionValue(ArrayValue):
-    """
-    Concrete class for union array elements.
-    """
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CUnionArray*> sp_array.get()
-
-    cdef getitem(self, int64_t i):
-        cdef int child_id = self.ap.child_id(i)
-        cdef shared_ptr[CArray] child = self.ap.field(child_id)
-        cdef CDenseUnionArray* dense
-        if self.ap.mode() == _UnionMode_SPARSE:
-            return box_scalar(self.type[child_id].type, child, i)
-        else:
-            dense = <CDenseUnionArray*> self.ap
-            return box_scalar(self.type[child_id].type, child,
-                              dense.value_offset(i))
 
-    def as_py(self):
-        """
-        Return this value as a Python object.
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
-        The exact type depends on the underlying union member.
-        """
-        return self.getitem(self.index).as_py()
 
+cdef class LargeListScalar(ListScalar):
+    pass
 
-cdef class FixedSizeBinaryValue(ArrayValue):
-    """
-    Concrete class for fixed-size binary array elements.
-    """
 
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            CFixedSizeBinaryArray* ap
-            CFixedSizeBinaryType* ap_type
-            int32_t length
-            const char* data
-        ap = <CFixedSizeBinaryArray*> self.sp_array.get()
-        ap_type = <CFixedSizeBinaryType*> ap.type().get()
-        length = ap_type.byte_width()
-        data = <const char*> ap.GetValue(self.index)
-        return cp.PyBytes_FromStringAndSize(data, length)
-
-
-cdef class StructValue(ArrayValue):
+cdef class StructScalar(Scalar, collections.abc.Mapping):
     """
-    Concrete class for struct array elements.
+    Concrete class for struct scalars.
     """
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CStructArray*> sp_array.get()
-
-    def __getitem__(self, key):
-        """
-        Return the child value for the given field name.
-        """
-        cdef:
-            CStructType* type
-            int index
-
-        type = <CStructType*> self.type.type
-        index = type.GetFieldIndex(tobytes(key))
-
-        if index < 0:
-            raise KeyError(key)
-
-        return pyarrow_wrap_array(self.ap.field(index))[self.index]
+    def __len__(self):
+        cdef CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+        return sp.value.size()
 
-    def as_py(self):
-        """
-        Return this value as a Python dict.
-        """
+    def __iter__(self):
         cdef:
-            vector[shared_ptr[CField]] child_fields = self.type.type.fields()
-
-        wrapped_arrays = [pyarrow_wrap_array(self.ap.field(i))
-                          for i in range(self.ap.num_fields())]
-        child_names = [child.get().name() for child in child_fields]
-        # Return the struct as a dict
-        return {
-            frombytes(name): child_array[self.index].as_py()
-            for name, child_array in zip(child_names, wrapped_arrays)
-        }
-
-
-cdef class DictionaryValue(ArrayValue):
-    """
-    Concrete class for dictionary-encoded array elements.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python object.
-
-        The exact type depends on the dictionary value type.
-        """
-        return self.dictionary_value.as_py()
-
-    @property
-    def index_value(self):
-        """
-        Return this value's underlying index as a ArrayValue of the right
-        signed integer type.
-        """
-        cdef CDictionaryArray* darr = <CDictionaryArray*>(self.sp_array.get())
-        indices = pyarrow_wrap_array(darr.indices())
-        return indices[self.index]
-
-    @property
-    def dictionary_value(self):
-        """
-        Return this value's underlying dictionary value as a ArrayValue.
-        """
-        cdef CDictionaryArray* darr = <CDictionaryArray*>(self.sp_array.get())
-        dictionary = pyarrow_wrap_array(darr.dictionary())
-        return dictionary[self.index_value.as_py()]
-
-
-cdef dict _array_value_classes = {
-    _Type_BOOL: BooleanValue,
-    _Type_UINT8: UInt8Value,
-    _Type_UINT16: UInt16Value,
-    _Type_UINT32: UInt32Value,
-    _Type_UINT64: UInt64Value,
-    _Type_INT8: Int8Value,
-    _Type_INT16: Int16Value,
-    _Type_INT32: Int32Value,
-    _Type_INT64: Int64Value,
-    _Type_DATE32: Date32Value,
-    _Type_DATE64: Date64Value,
-    _Type_TIME32: Time32Value,
-    _Type_TIME64: Time64Value,
-    _Type_TIMESTAMP: TimestampValue,
-    _Type_DURATION: DurationValue,
-    _Type_HALF_FLOAT: HalfFloatValue,
-    _Type_FLOAT: FloatValue,
-    _Type_DOUBLE: DoubleValue,
-    _Type_LIST: ListValue,
-    _Type_LARGE_LIST: LargeListValue,
-    _Type_MAP: MapValue,
-    _Type_FIXED_SIZE_LIST: FixedSizeListValue,
-    _Type_SPARSE_UNION: UnionValue,
-    _Type_DENSE_UNION: UnionValue,
-    _Type_BINARY: BinaryValue,
-    _Type_STRING: StringValue,
-    _Type_LARGE_BINARY: LargeBinaryValue,
-    _Type_LARGE_STRING: LargeStringValue,
-    _Type_FIXED_SIZE_BINARY: FixedSizeBinaryValue,
-    _Type_DECIMAL: DecimalValue,
-    _Type_STRUCT: StructValue,
-    _Type_DICTIONARY: DictionaryValue,
-}
-
-cdef class ScalarValue(Scalar):
-    """
-    The base class for scalars.
-    """
-
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly."
-                        .format(self.__class__.__name__))
-
-    cdef void init(self, const shared_ptr[CScalar]& sp_scalar):
-        self.sp_scalar = sp_scalar
-
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+            CStructType* dtype = <CStructType*> sp.type.get()
+            vector[shared_ptr[CField]] fields = dtype.fields()
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+        if sp.is_valid:
+            for i in range(dtype.num_fields()):
+                yield frombytes(fields[i].get().name())
 
-    def __eq__(self, other):
-        if hasattr(self, 'as_py'):
-            if isinstance(other, ScalarValue):
-                other = other.as_py()
-            return self.as_py() == other
+    def __contains__(self, key):
+        try:
+            self[key]
+        except (KeyError, IndexError):
+            return False
         else:
-            raise NotImplemented(
-                "Cannot compare scalars that don't support as_py()")
-
-    def __hash__(self):
-        return hash(self.as_py())
-
-
-cdef class NullScalar(ScalarValue):
-    """
-    Concrete class for null scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python None.
-        """
-        return None
-
-
-cdef class BooleanScalar(ScalarValue):
-    """
-    Concrete class for boolean scalars.
-    """
+            return True
 
-    def as_py(self):
-        """
-        Return this value as a Python bool.
+    def __getitem__(self, key):
         """
-        cdef CBooleanScalar* sp = <CBooleanScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
+        Return the child value for the given field.
 
-cdef class UInt8Scalar(ScalarValue):
-    """
-    Concrete class for uint8 scalars.
-    """
+        Parameters
+        ----------
+        index : Union[int, str]
+            Index / position or name of the field.
 
-    def as_py(self):
-        """
-        Return this value as a Python int.
+        Returns
+        -------
+        result : Scalar
         """
-        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        cdef:
+            CFieldRef ref
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
 
+        if isinstance(key, (bytes, str)):
+            ref = CFieldRef(<c_string> tobytes(key))
+        elif isinstance(key, int):
+            ref = CFieldRef(<int> key)
+        else:
+            raise TypeError('Expected integer or string index')
 
-cdef class Int8Scalar(ScalarValue):
-    """
-    Concrete class for int8 scalars.
-    """
+        try:
+            return Scalar.wrap(GetResultValue(sp.field(ref)))
+        except ArrowInvalid:
+            if isinstance(key, int):
+                raise IndexError(key)
+            else:
+                raise KeyError(key)
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this value as a Python dict.
         """
-        cdef CInt8Scalar* sp = <CInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        if self.is_valid:
+            return {k: v.as_py() for k, v in self.items()}
+        else:
+            return None
 
 
-cdef class UInt16Scalar(ScalarValue):
+cdef class MapScalar(ListScalar):
     """
-    Concrete class for uint16 scalars.
+    Concrete class for map scalars.
     """
 
-    def as_py(self):
+    def __getitem__(self, i):
         """
-        Return this value as a Python int.
+        Return the value at the given index.
         """
-        cdef CUInt16Scalar* sp = <CUInt16Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int16Scalar(ScalarValue):
-    """
-    Concrete class for int16 scalars.
-    """
+        arr = self.values
+        if arr is None:
+            raise IndexError(i)
+        dct = arr[_normalize_index(i, len(arr))]
+        return (dct['key'], dct['value'])
 
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python int.
+        Iterate over this element's values.
         """
-        cdef CInt16Scalar* sp = <CInt16Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt32Scalar(ScalarValue):
-    """
-    Concrete class for uint32 scalars.
-    """
+        arr = self.values
+        if arr is None:
+            return iter(zip(arr.field('key'), arr.field('value')))
+        else:
+            raise StopIteration
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this value as a Python list.
         """
-        cdef CUInt32Scalar* sp = <CUInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        arr = self.values
+        if arr is not None:
+            return list(zip(arr.field('key'), arr.field('value')))
+        else:
+            return None
 
 
-cdef class Int32Scalar(ScalarValue):
+cdef class DictionaryScalar(Scalar):
     """
-    Concrete class for int32 scalars.
+    Concrete class for dictionary-encoded scalars.
     """
 
-    def as_py(self):
+    @property
+    def index(self):
         """
-        Return this value as a Python int.
+        Return this value's underlying index as a scalar.
         """
-        cdef CInt32Scalar* sp = <CInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+        return Scalar.wrap(sp.value.index)
 
-
-cdef class UInt64Scalar(ScalarValue):
-    """
-    Concrete class for uint64 scalars.
-    """
-
-    def as_py(self):
+    @property
+    def value(self):
         """
-        Return this value as a Python int.
+        Return the encoded value as a scalar.
         """
-        cdef CUInt64Scalar* sp = <CUInt64Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        # TODO(kszucs): optimize it to spare uneccessary python object boxing
+        return self.dictionary[self.index.as_py()] if self.is_valid else None
 
-
-cdef class Int64Scalar(ScalarValue):
-    """
-    Concrete class for int64 scalars.
-    """
+    @property
+    def dictionary(self):
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_array(sp.value.dictionary)
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this encoded value as a Python object.
         """
-        cdef CInt64Scalar* sp = <CInt64Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
+        value = self.value
+        return None if value is None else value.as_py()
 
-cdef class FloatScalar(ScalarValue):
-    """
-    Concrete class for float scalars.
-    """
+    @property
+    def index_value(self):
+        warnings.warn("`dictionary_value` property is deprecated as of 1.0.0"
+                      "please use the `value` property instead",
+                      FutureWarning)
+        index = self.index
+        return None if index is None else self.index

Review comment:
       Updated.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,748 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
-    """
-
-
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
+    The base class for scalars.
     """
-    # TODO rename this NullValue?
-
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
-
-        self.type = null()
-
-    def __repr__(self):
-        return 'NULL'
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-    def __eq__(self, other):
-        return NA
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
 
-_NULL = NA = NullType()
+        if type_id == _Type_NA:
+            return _NULL
 
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
 
-cdef class ArrayValue(Scalar):
-    """
-    The base class for non-null array elements.
-    """
+        return self
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
 
-    cdef void init(self, DataType type, const shared_ptr[CArray]& sp_array,
-                   int64_t index):
-        self.type = type
-        self.index = index
-        self._set_array(sp_array)
+    @property
+    def type(self):
+        """
+        Data type of the Scalar object.
+        """
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    @property
+    def is_valid(self):
+        """
+        Holds a valid (non-null) value.
+        """
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
     def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        if hasattr(self, 'as_py'):
-            if isinstance(other, ArrayValue):
-                other = other.as_py()
-            return self.as_py() == other
-        else:
-            raise NotImplementedError(
-                "Cannot compare Arrow values that don't support as_py()")
+        try:
+            if not isinstance(other, Scalar):
+                other = scalar(other, type=self.type)
+            return self.equals(other)
+        except (TypeError, ValueError, ArrowInvalid):
+            return NotImplemented
 
     def __hash__(self):
-        return hash(self.as_py())
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
+    def as_py(self):
+        raise NotImplementedError()
 
-cdef class BooleanValue(ArrayValue):
-    """
-    Concrete class for boolean array elements.
-    """
 
-    def as_py(self):
-        """
-        Return this value as a Python bool.
-        """
-        cdef CBooleanArray* ap = <CBooleanArray*> self.sp_array.get()
-        return ap.Value(self.index)
+_NULL = NA = None
 
 
-cdef class Int8Value(ArrayValue):
+cdef class NullScalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for null scalars.
     """
 
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt8Array* ap = <CInt8Array*> self.sp_array.get()
-        return ap.Value(self.index)
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise RuntimeError('Cannot create multiple NullScalar instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
+    def __init__(self):
+        pass
 
-cdef class UInt8Value(ArrayValue):
-    """
-    Concrete class for uint8 array elements.
-    """
+    def __eq__(self, other):
+        return NA
+
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this value as a Python None.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> self.sp_array.get()
-        return ap.Value(self.index)
-
+        return None
 
-cdef class Int16Value(ArrayValue):
-    """
-    Concrete class for int16 array elements.
-    """
 
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt16Array* ap = <CInt16Array*> self.sp_array.get()
-        return ap.Value(self.index)
+_NULL = NA = NullScalar()
 
 
-cdef class UInt16Value(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this value as a Python bool.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CBooleanScalar* sp = <CBooleanScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int32Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt32Array* ap = <CInt32Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt32Array* ap = <CUInt32Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CInt8Scalar* sp = <CInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt64Array* ap = <CInt64Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CUInt16Scalar* sp = <CUInt16Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt64Array* ap = <CUInt64Array*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CInt16Scalar* sp = <CInt16Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for uint32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python int.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
-
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+        cdef CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for int32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python int.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for uint64 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python int.
         """
-        cdef:
-            CTime32Array* ap = <CTime32Array*> self.sp_array.get()
-            CTime32Type* dtype = <CTime32Type*> ap.type().get()
-
-        if dtype.unit() == TimeUnit_SECOND:
-            delta = datetime.timedelta(seconds=ap.Value(self.index))
-            return (datetime.datetime(1970, 1, 1) + delta).time()
-        else:
-            return _box_time_milli(ap.Value(self.index))
+        cdef CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for int64 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python int.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
-
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
-        else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
-
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
-
-
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
-
-
-cdef dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
-
-
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
-    if tzinfo is not None:
-        if not isinstance(tzinfo, datetime.tzinfo):
-            tzinfo = string_to_tzinfo(tzinfo)
-        dt = tzinfo.fromutc(dt)
-    return dt
-
-
-cdef _datetime_from_seconds(int64_t v):
-    return datetime.datetime(1970, 1, 1) + datetime.timedelta(seconds=v)
-
-
-def _nanoseconds_to_datetime_safe(v, tzinfo):
-    if v % 1000 != 0:
-        raise ValueError("Nanosecond timestamp {} is not safely convertible "
-                         " to microseconds to convert to datetime.datetime."
-                         " Install pandas to return as Timestamp with "
-                         " nanosecond support or access the .value attribute.")
-    v = v // 1000
-    micros = v % 1_000_000
-
-    dt = _datetime_from_seconds(v // 1_000_000)
-    return _add_micros_maybe_localize(dt, micros, tzinfo)
-
-
-def _microseconds_to_datetime(v, tzinfo):
-    micros = v % 1_000_000
-    dt = _datetime_from_seconds(v // 1_000_000)
-    return _add_micros_maybe_localize(dt, micros, tzinfo)
-
-
-def _millis_to_datetime(v, tzinfo):
-    millis = v % 1_000
-    dt = _datetime_from_seconds(v // 1000)
-    return _add_micros_maybe_localize(dt, millis * 1000, tzinfo)
-
-
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
-
-
-def _datetime_conversion_functions():
-    global _datetime_conversion_initialized
-    if _datetime_conversion_initialized:
-        return _DATETIME_CONVERSION_FUNCTIONS
-
-    _DATETIME_CONVERSION_FUNCTIONS.update({
-        TimeUnit_SECOND: _seconds_to_datetime,
-        TimeUnit_MILLI: _millis_to_datetime,
-        TimeUnit_MICRO: _microseconds_to_datetime,
-        TimeUnit_NANO: _nanoseconds_to_datetime_safe
-    })
-
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
-
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        cdef CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for float scalars.
     """
 
-    @property
-    def value(self):
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
-        return ap.Value(self.index)
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if isinstance(other, Scalar):
+            other = other.as_py()
+        return self.as_py() == other
 
     def as_py(self):
         """
-        Return this value as a Pandas Timestamp instance (if available),
-        otherwise as a Python datetime.timedelta instance.
+        Return this value as a Python float.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
-
-        value = self.value
-
-        if not dtype.timezone().empty():
-            tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
-        else:
-            tzinfo = None
-
-        try:
-            converter = _datetime_conversion_functions()[dtype.unit()]
-        except KeyError:
-            raise ValueError(
-                'Cannot convert nanosecond timestamps without pandas'
-            )
-        return converter(value, tzinfo=tzinfo)
-
-
-cdef dict _TIMEDELTA_CONVERSION_FUNCTIONS = {}
-
-
-def _nanoseconds_to_timedelta_safe(v):
-    if v % 1000 != 0:
-        raise ValueError(
-            "Nanosecond duration {} is not safely convertible to microseconds "
-            "to convert to datetime.timedelta. Install pandas to return as "
-            "Timedelta with nanosecond support or access the .value "
-            "attribute.".format(v))
-    micros = v // 1000
-
-    return datetime.timedelta(microseconds=micros)
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
 
-def _timedelta_conversion_functions():
-    if _TIMEDELTA_CONVERSION_FUNCTIONS:
-        return _TIMEDELTA_CONVERSION_FUNCTIONS
-
-    _TIMEDELTA_CONVERSION_FUNCTIONS.update({
-        TimeUnit_SECOND: lambda v: datetime.timedelta(seconds=v),
-        TimeUnit_MILLI: lambda v: datetime.timedelta(milliseconds=v),
-        TimeUnit_MICRO: lambda v: datetime.timedelta(microseconds=v),
-        TimeUnit_NANO: _nanoseconds_to_timedelta_safe
-    })
-
-    try:
-        import pandas as pd
-        _TIMEDELTA_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda v: pd.Timedelta(v, unit='ns')
-        )
-    except ImportError:
-        pass
-
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
-
-
-cdef class DurationValue(ArrayValue):
+cdef class FloatScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for float scalars.
     """
 
-    @property
-    def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
-
     def as_py(self):
         """
-        Return this value as a Pandas Timestamp instance (if available),
-        otherwise as a Python datetime.timedelta instance.
+        Return this value as a Python float.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
-
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class HalfFloatValue(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python float.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class FloatValue(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for float32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python Decimal.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef:
+            CDecimal128Scalar* sp = <CDecimal128Scalar*> self.wrapped.get()
+            CDecimal128Type* dtype = <CDecimal128Type*> sp.type.get()
+        if sp.is_valid:
+            return _pydecimal.Decimal(
+                frombytes(sp.value.ToString(dtype.scale()))
+            )
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for float64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
+
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
+        else:
+            return None
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class Date64Scalar(Scalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for date64 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        cdef CDate64Scalar* sp = <CDate64Scalar*> self.wrapped.get()
 
+        if sp.is_valid:
+            return (
+                datetime.date(1970, 1, 1) +
+                datetime.timedelta(days=sp.value / 86400000)
+            )
+        else:
+            return None
 
-cdef class StringValue(ArrayValue):
-    """
-    Concrete class for string (utf8) array elements.
-    """
 
-    def as_py(self):
-        """
-        Return this value as a Python unicode string.
-        """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+def _datetime_from_int(int64_t value, TimeUnit unit, tzinfo=None):
+    if unit == TimeUnit_SECOND:
+        delta = datetime.timedelta(seconds=value)
+    elif unit == TimeUnit_MILLI:
+        delta = datetime.timedelta(milliseconds=value)
+    elif unit == TimeUnit_MICRO:
+        delta = datetime.timedelta(microseconds=value)
+    else:
+        # TimeUnit_NANO: prefer pandas timestamps if available
+        if _pandas_api.have_pandas:
+            return _pandas_api.pd.Timestamp(value, tz=tzinfo, unit='ns')
+        # otherwise safely truncate to microsecond resolution datetime
+        if value % 1000 != 0:
+            raise ValueError(
+                "Nanosecond resolution temporal type {} is not safely "
+                "convertible to microseconds to convert to datetime.datetime. "
+                "Install pandas to return as Timestamp with nanosecond "
+                "support or access the .value attribute.".format(value)
+            )
+        delta = datetime.timedelta(microseconds=value // 1000)
 
-    def as_buffer(self):
-        """
-        Return a view over this value as a Buffer object.
-        """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
+    if tzinfo is not None:
+        dt = tzinfo.fromutc(dt)
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+    return dt
 
 
-cdef class LargeStringValue(ArrayValue):
+cdef class Time32Scalar(Scalar):
     """
-    Concrete class for large string (utf8) array elements.
+    Concrete class for time32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python unicode string.
-        """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
-
-    def as_buffer(self):
-        """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python datetime.timedelta instance.
         """
         cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class Time64Scalar(Scalar):
     """
-    Concrete class for variable-sized binary array elements.
+    Concrete class for time64 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            const uint8_t* ptr
-            int32_t length
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
-
-    def as_buffer(self):
-        """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python datetime.timedelta instance.
         """
         cdef:
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class LargeBinaryValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for large variable-sized binary array elements.
+    Concrete class for timestamp scalars.
     """
 
+    @property
+    def value(self):
+        cdef CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
+
     def as_py(self):
         """
-        Return this value as a Python bytes object.
+        Return this value as a Pandas Timestamp instance (if available),
+        otherwise as a Python datetime.timedelta instance.
         """
         cdef:
-            const uint8_t* ptr
-            int64_t length
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
+        if not sp.is_valid:
+            return None
 
-    def as_buffer(self):
-        """
-        Return a view over this value as a Buffer object.
-        """
-        cdef:
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        if not dtype.timezone().empty():
+            tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+        else:
+            tzinfo = None
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        return _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
 
-cdef class ListValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for list array elements.
+    Concrete class for duration scalars.
     """
 
-    def __len__(self):
-        """
-        Return the number of values.
-        """
-        return self.length()
-
-    def __getitem__(self, i):
-        """
-        Return the value at the given index.
-        """
-        return self.getitem(_normalize_index(i, self.length()))
-
-    def __iter__(self):
-        """
-        Iterate over this element's values.
-        """
-        for i in range(len(self)):
-            yield self.getitem(i)
-        raise StopIteration
-
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CListArray*> sp_array.get()
-        self.value_type = pyarrow_wrap_data_type(self.ap.value_type())
-
-    cdef getitem(self, int64_t i):
-        cdef int64_t j = self.ap.value_offset(self.index) + i
-        return box_scalar(self.value_type, self.ap.values(), j)
-
-    cdef int64_t length(self):
-        return self.ap.value_length(self.index)
+    @property
+    def value(self):
+        cdef CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python list.
+        Return this value as a Pandas Timestamp instance (if available),
+        otherwise as a Python datetime.timedelta instance.
         """
         cdef:
-            int64_t j
-            list result = []
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
 
-        for j in range(len(self)):
-            result.append(self.getitem(j).as_py())
+        if not sp.is_valid:
+            return None
 
-        return result
+        if unit == TimeUnit_SECOND:
+            return datetime.timedelta(seconds=sp.value)
+        elif unit == TimeUnit_MILLI:
+            return datetime.timedelta(milliseconds=sp.value)
+        elif unit == TimeUnit_MICRO:
+            return datetime.timedelta(microseconds=sp.value)
+        else:
+            # TimeUnit_NANO: prefer pandas timestamps if available
+            if _pandas_api.have_pandas:
+                return _pandas_api.pd.Timedelta(sp.value, unit='ns')
+            # otherwise safely truncate to microsecond resolution timedelta
+            if sp.value % 1000 != 0:
+                raise ValueError(
+                    "Nanosecond duration {} is not safely convertible to "
+                    "microseconds to convert to datetime.timedelta. Install "
+                    "pandas to return as Timedelta with nanosecond support or "
+                    "access the .value attribute.".format(sp.value)
+                )
+            return datetime.timedelta(microseconds=sp.value // 1000)
 
 
-cdef class LargeListValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for large list array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def __len__(self):
-        """
-        Return the number of values.
-        """
-        return self.length()
-
-    def __getitem__(self, i):
+    def as_buffer(self):
         """
-        Return the value at the given index.
+        Return a view over this value as a Buffer object.
         """
-        return self.getitem(_normalize_index(i, self.length()))
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
-    def __iter__(self):
+    def as_py(self):
         """
-        Iterate over this element's values.
+        Return this value as a Python bytes.
         """
-        for i in range(len(self)):
-            yield self.getitem(i)
-        raise StopIteration
+        buffer = self.as_buffer()
+        return None if buffer is None else buffer.to_pybytes()
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CLargeListArray*> sp_array.get()
-        self.value_type = pyarrow_wrap_data_type(self.ap.value_type())
 
-    cdef getitem(self, int64_t i):
-        cdef int64_t j = self.ap.value_offset(self.index) + i
-        return box_scalar(self.value_type, self.ap.values(), j)
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    cdef int64_t length(self):
-        return self.ap.value_length(self.index)
 
-    def as_py(self):
-        """
-        Return this value as a Python list.
-        """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            result.append(self.getitem(j).as_py())
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
-        return result
 
-
-cdef class MapValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for map array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
-    def __len__(self):
-        """
-        Return the number of values.
-        """
-        return self.length()
-
-    def __getitem__(self, i):
-        """
-        Return the value at the given index.
-        """
-        return self.getitem(_normalize_index(i, self.length()))
-
-    def __iter__(self):
-        """
-        Iterate over this element's values.
-        """
-        for i in range(len(self)):
-            yield self.getitem(i)
-        raise StopIteration
-
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CMapArray*> sp_array.get()
-        self.key_type = pyarrow_wrap_data_type(self.ap.map_type().key_type())
-        self.item_type = pyarrow_wrap_data_type(self.ap.map_type().item_type())
-
-    cdef getitem(self, int64_t i):
-        cdef int64_t j = self.ap.value_offset(self.index) + i
-        return (box_scalar(self.key_type, self.ap.keys(), j),
-                box_scalar(self.item_type, self.ap.items(), j))
-
-    cdef int64_t length(self):
-        return self.ap.value_length(self.index)
-
     def as_py(self):
         """
-        Return this value as a Python list of tuples, each containing a
-        key and item.
+        Return this value as a Python string.
         """
-        cdef:
-            int64_t j
-            list result = []
+        buffer = self.as_buffer()
+        return None if buffer is None else str(buffer, 'utf8')
 
-        for j in range(len(self)):
-            key, item = self.getitem(j)
-            result.append((key.as_py(), item.as_py()))
 
-        return result
+cdef class LargeStringScalar(StringScalar):
+    pass
 
 
-cdef class FixedSizeListValue(ArrayValue):
+cdef class ListScalar(Scalar):
     """
-    Concrete class for fixed size list array elements.
+    Concrete class for list-like scalars.
     """
 
+    @property
+    def values(self):
+        cdef CListScalar* sp = <CListScalar*> self.wrapped.get()
+        if sp.is_valid:
+            return pyarrow_wrap_array(sp.value)
+        else:
+            return None
+
     def __len__(self):
         """
         Return the number of values.
         """
-        return self.length()
+        return len(self.values)
 
     def __getitem__(self, i):
         """
         Return the value at the given index.
         """
-        return self.getitem(_normalize_index(i, self.length()))
+        return self.values[_normalize_index(i, len(self))]
 
     def __iter__(self):
         """
         Iterate over this element's values.
         """
-        for i in range(len(self)):
-            yield self.getitem(i)
-        raise StopIteration
-
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CFixedSizeListArray*> sp_array.get()
-        self.value_type = pyarrow_wrap_data_type(self.ap.value_type())
-
-    cdef getitem(self, int64_t i):
-        cdef int64_t j = self.ap.value_offset(self.index) + i
-        return box_scalar(self.value_type, self.ap.values(), j)
-
-    cdef int64_t length(self):
-        return self.ap.value_length(self.index)
+        return iter(self.values)
 
     def as_py(self):
         """
         Return this value as a Python list.
         """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            result.append(self.getitem(j).as_py())
-
-        return result
-
-
-cdef class UnionValue(ArrayValue):
-    """
-    Concrete class for union array elements.
-    """
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CUnionArray*> sp_array.get()
-
-    cdef getitem(self, int64_t i):
-        cdef int child_id = self.ap.child_id(i)
-        cdef shared_ptr[CArray] child = self.ap.field(child_id)
-        cdef CDenseUnionArray* dense
-        if self.ap.mode() == _UnionMode_SPARSE:
-            return box_scalar(self.type[child_id].type, child, i)
-        else:
-            dense = <CDenseUnionArray*> self.ap
-            return box_scalar(self.type[child_id].type, child,
-                              dense.value_offset(i))
 
-    def as_py(self):
-        """
-        Return this value as a Python object.
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
-        The exact type depends on the underlying union member.
-        """
-        return self.getitem(self.index).as_py()
 
+cdef class LargeListScalar(ListScalar):
+    pass
 
-cdef class FixedSizeBinaryValue(ArrayValue):
-    """
-    Concrete class for fixed-size binary array elements.
-    """
 
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            CFixedSizeBinaryArray* ap
-            CFixedSizeBinaryType* ap_type
-            int32_t length
-            const char* data
-        ap = <CFixedSizeBinaryArray*> self.sp_array.get()
-        ap_type = <CFixedSizeBinaryType*> ap.type().get()
-        length = ap_type.byte_width()
-        data = <const char*> ap.GetValue(self.index)
-        return cp.PyBytes_FromStringAndSize(data, length)
-
-
-cdef class StructValue(ArrayValue):
+cdef class StructScalar(Scalar, collections.abc.Mapping):
     """
-    Concrete class for struct array elements.
+    Concrete class for struct scalars.
     """
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
-        self.ap = <CStructArray*> sp_array.get()
-
-    def __getitem__(self, key):
-        """
-        Return the child value for the given field name.
-        """
-        cdef:
-            CStructType* type
-            int index
-
-        type = <CStructType*> self.type.type
-        index = type.GetFieldIndex(tobytes(key))
-
-        if index < 0:
-            raise KeyError(key)
-
-        return pyarrow_wrap_array(self.ap.field(index))[self.index]
+    def __len__(self):
+        cdef CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+        return sp.value.size()
 
-    def as_py(self):
-        """
-        Return this value as a Python dict.
-        """
+    def __iter__(self):
         cdef:
-            vector[shared_ptr[CField]] child_fields = self.type.type.fields()
-
-        wrapped_arrays = [pyarrow_wrap_array(self.ap.field(i))
-                          for i in range(self.ap.num_fields())]
-        child_names = [child.get().name() for child in child_fields]
-        # Return the struct as a dict
-        return {
-            frombytes(name): child_array[self.index].as_py()
-            for name, child_array in zip(child_names, wrapped_arrays)
-        }
-
-
-cdef class DictionaryValue(ArrayValue):
-    """
-    Concrete class for dictionary-encoded array elements.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python object.
-
-        The exact type depends on the dictionary value type.
-        """
-        return self.dictionary_value.as_py()
-
-    @property
-    def index_value(self):
-        """
-        Return this value's underlying index as a ArrayValue of the right
-        signed integer type.
-        """
-        cdef CDictionaryArray* darr = <CDictionaryArray*>(self.sp_array.get())
-        indices = pyarrow_wrap_array(darr.indices())
-        return indices[self.index]
-
-    @property
-    def dictionary_value(self):
-        """
-        Return this value's underlying dictionary value as a ArrayValue.
-        """
-        cdef CDictionaryArray* darr = <CDictionaryArray*>(self.sp_array.get())
-        dictionary = pyarrow_wrap_array(darr.dictionary())
-        return dictionary[self.index_value.as_py()]
-
-
-cdef dict _array_value_classes = {
-    _Type_BOOL: BooleanValue,
-    _Type_UINT8: UInt8Value,
-    _Type_UINT16: UInt16Value,
-    _Type_UINT32: UInt32Value,
-    _Type_UINT64: UInt64Value,
-    _Type_INT8: Int8Value,
-    _Type_INT16: Int16Value,
-    _Type_INT32: Int32Value,
-    _Type_INT64: Int64Value,
-    _Type_DATE32: Date32Value,
-    _Type_DATE64: Date64Value,
-    _Type_TIME32: Time32Value,
-    _Type_TIME64: Time64Value,
-    _Type_TIMESTAMP: TimestampValue,
-    _Type_DURATION: DurationValue,
-    _Type_HALF_FLOAT: HalfFloatValue,
-    _Type_FLOAT: FloatValue,
-    _Type_DOUBLE: DoubleValue,
-    _Type_LIST: ListValue,
-    _Type_LARGE_LIST: LargeListValue,
-    _Type_MAP: MapValue,
-    _Type_FIXED_SIZE_LIST: FixedSizeListValue,
-    _Type_SPARSE_UNION: UnionValue,
-    _Type_DENSE_UNION: UnionValue,
-    _Type_BINARY: BinaryValue,
-    _Type_STRING: StringValue,
-    _Type_LARGE_BINARY: LargeBinaryValue,
-    _Type_LARGE_STRING: LargeStringValue,
-    _Type_FIXED_SIZE_BINARY: FixedSizeBinaryValue,
-    _Type_DECIMAL: DecimalValue,
-    _Type_STRUCT: StructValue,
-    _Type_DICTIONARY: DictionaryValue,
-}
-
-cdef class ScalarValue(Scalar):
-    """
-    The base class for scalars.
-    """
-
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly."
-                        .format(self.__class__.__name__))
-
-    cdef void init(self, const shared_ptr[CScalar]& sp_scalar):
-        self.sp_scalar = sp_scalar
-
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+            CStructType* dtype = <CStructType*> sp.type.get()
+            vector[shared_ptr[CField]] fields = dtype.fields()
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+        if sp.is_valid:
+            for i in range(dtype.num_fields()):
+                yield frombytes(fields[i].get().name())
 
-    def __eq__(self, other):
-        if hasattr(self, 'as_py'):
-            if isinstance(other, ScalarValue):
-                other = other.as_py()
-            return self.as_py() == other
+    def __contains__(self, key):
+        try:
+            self[key]
+        except (KeyError, IndexError):
+            return False
         else:
-            raise NotImplemented(
-                "Cannot compare scalars that don't support as_py()")
-
-    def __hash__(self):
-        return hash(self.as_py())
-
-
-cdef class NullScalar(ScalarValue):
-    """
-    Concrete class for null scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python None.
-        """
-        return None
-
-
-cdef class BooleanScalar(ScalarValue):
-    """
-    Concrete class for boolean scalars.
-    """
+            return True
 
-    def as_py(self):
-        """
-        Return this value as a Python bool.
+    def __getitem__(self, key):
         """
-        cdef CBooleanScalar* sp = <CBooleanScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
+        Return the child value for the given field.
 
-cdef class UInt8Scalar(ScalarValue):
-    """
-    Concrete class for uint8 scalars.
-    """
+        Parameters
+        ----------
+        index : Union[int, str]
+            Index / position or name of the field.
 
-    def as_py(self):
-        """
-        Return this value as a Python int.
+        Returns
+        -------
+        result : Scalar
         """
-        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        cdef:
+            CFieldRef ref
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
 
+        if isinstance(key, (bytes, str)):
+            ref = CFieldRef(<c_string> tobytes(key))
+        elif isinstance(key, int):
+            ref = CFieldRef(<int> key)
+        else:
+            raise TypeError('Expected integer or string index')
 
-cdef class Int8Scalar(ScalarValue):
-    """
-    Concrete class for int8 scalars.
-    """
+        try:
+            return Scalar.wrap(GetResultValue(sp.field(ref)))
+        except ArrowInvalid:
+            if isinstance(key, int):
+                raise IndexError(key)
+            else:
+                raise KeyError(key)
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this value as a Python dict.
         """
-        cdef CInt8Scalar* sp = <CInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        if self.is_valid:
+            return {k: v.as_py() for k, v in self.items()}
+        else:
+            return None
 
 
-cdef class UInt16Scalar(ScalarValue):
+cdef class MapScalar(ListScalar):
     """
-    Concrete class for uint16 scalars.
+    Concrete class for map scalars.
     """
 
-    def as_py(self):
+    def __getitem__(self, i):
         """
-        Return this value as a Python int.
+        Return the value at the given index.
         """
-        cdef CUInt16Scalar* sp = <CUInt16Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int16Scalar(ScalarValue):
-    """
-    Concrete class for int16 scalars.
-    """
+        arr = self.values
+        if arr is None:
+            raise IndexError(i)
+        dct = arr[_normalize_index(i, len(arr))]
+        return (dct['key'], dct['value'])
 
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python int.
+        Iterate over this element's values.
         """
-        cdef CInt16Scalar* sp = <CInt16Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt32Scalar(ScalarValue):
-    """
-    Concrete class for uint32 scalars.
-    """
+        arr = self.values
+        if arr is None:
+            return iter(zip(arr.field('key'), arr.field('value')))
+        else:
+            raise StopIteration
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this value as a Python list.
         """
-        cdef CUInt32Scalar* sp = <CUInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        arr = self.values
+        if arr is not None:
+            return list(zip(arr.field('key'), arr.field('value')))
+        else:
+            return None
 
 
-cdef class Int32Scalar(ScalarValue):
+cdef class DictionaryScalar(Scalar):
     """
-    Concrete class for int32 scalars.
+    Concrete class for dictionary-encoded scalars.
     """
 
-    def as_py(self):
+    @property
+    def index(self):
         """
-        Return this value as a Python int.
+        Return this value's underlying index as a scalar.
         """
-        cdef CInt32Scalar* sp = <CInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+        return Scalar.wrap(sp.value.index)
 
-
-cdef class UInt64Scalar(ScalarValue):
-    """
-    Concrete class for uint64 scalars.
-    """
-
-    def as_py(self):
+    @property
+    def value(self):
         """
-        Return this value as a Python int.
+        Return the encoded value as a scalar.
         """
-        cdef CUInt64Scalar* sp = <CUInt64Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+        # TODO(kszucs): optimize it to spare uneccessary python object boxing
+        return self.dictionary[self.index.as_py()] if self.is_valid else None
 
-
-cdef class Int64Scalar(ScalarValue):
-    """
-    Concrete class for int64 scalars.
-    """
+    @property
+    def dictionary(self):
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_array(sp.value.dictionary)
 
     def as_py(self):
         """
-        Return this value as a Python int.
+        Return this encoded value as a Python object.
         """
-        cdef CInt64Scalar* sp = <CInt64Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
+        value = self.value
+        return None if value is None else value.as_py()
 
-cdef class FloatScalar(ScalarValue):
-    """
-    Concrete class for float scalars.
-    """
+    @property
+    def index_value(self):
+        warnings.warn("`dictionary_value` property is deprecated as of 1.0.0"
+                      "please use the `value` property instead",
+                      FutureWarning)
+        index = self.index
+        return None if index is None else self.index
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CFloatScalar* sp = <CFloatScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
+    @property
+    def dictionary_value(self):
+        warnings.warn("`dictionary_value` property is deprecated as of 1.0.0, "
+                      "please use the `value` property instead", FutureWarning)
+        value = self.value
+        return None if value is None else self.value

Review comment:
       Updated.




----------------------------------------------------------------
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.

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