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/06/30 13:48:46 UTC

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

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



##########
File path: python/pyarrow/lib.pxd
##########
@@ -179,101 +179,18 @@ cdef class Schema:
 
 
 cdef class Scalar:
-    cdef readonly:
-        DataType type
-
-
-cdef class NAType(Scalar):
-    pass
-
-
-cdef class ArrayValue(Scalar):
     cdef:
-        shared_ptr[CArray] sp_array
-        int64_t index
-
-    cdef void init(self, DataType type,
-                   const shared_ptr[CArray]& sp_array, int64_t index)
-
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array)
-
-cdef class ScalarValue(Scalar):
-    cdef:
-        shared_ptr[CScalar] sp_scalar
-
-    cdef void init(self, const shared_ptr[CScalar]& sp_scalar)
-
-cdef class Int8Value(ArrayValue):
-    pass
+        shared_ptr[CScalar] wrapped
 
+    cdef void init(self, const shared_ptr[CScalar]& wrapped)
 
-cdef class Int64Value(ArrayValue):
-    pass
-
-
-cdef class ListValue(ArrayValue):
-    cdef readonly:
-        DataType value_type
-
-    cdef:
-        CListArray* ap
-
-    cdef getitem(self, int64_t i)
-    cdef int64_t length(self)
-
-
-cdef class LargeListValue(ArrayValue):
-    cdef readonly:
-        DataType value_type
-
-    cdef:
-        CLargeListArray* ap
-
-    cdef getitem(self, int64_t i)
-    cdef int64_t length(self)
-
-
-cdef class MapValue(ArrayValue):
-    cdef readonly:
-        DataType key_type
-        DataType item_type
-
-    cdef:
-        CMapArray* ap
-
-    cdef getitem(self, int64_t i)
-    cdef int64_t length(self)
-
-
-cdef class FixedSizeListValue(ArrayValue):
-    cdef readonly:
-        DataType value_type
-
-    cdef:
-        CFixedSizeListArray* ap
-
-    cdef getitem(self, int64_t i)
-    cdef int64_t length(self)
-
-
-cdef class StructValue(ArrayValue):
-    cdef:
-        CStructArray* ap
-
-
-cdef class UnionValue(ArrayValue):
-    cdef:
-        CUnionArray* ap
-        list value_types
-
-    cdef getitem(self, int64_t i)
-
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped)
 
-cdef class StringValue(ArrayValue):
-    pass
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil
 
 
-cdef class FixedSizeBinaryValue(ArrayValue):
+cdef class NAType(Scalar):

Review comment:
       `NullScalar` perhaps... also, why does it have its own class?

##########
File path: python/pyarrow/includes/libarrow.pxd
##########
@@ -44,6 +44,11 @@ cdef extern from "arrow/util/key_value_metadata.h" namespace "arrow" nogil:
         c_bool Contains(const c_string& key) const
 
 
+cdef extern from "arrow/util/decimal.h" namespace "arrow" nogil:
+    cdef cppclass CDecimal128" arrow::Decimal128":
+        c_string ToString(int32_t scale) const

Review comment:
       Hmm, at some point we should be able to pretty-print scalars in C++.

##########
File path: python/pyarrow/includes/libarrow.pxd
##########
@@ -877,20 +893,60 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
     cdef cppclass CUInt64Scalar" arrow::UInt64Scalar"(CScalar):
         uint64_t value
 
+    cdef cppclass CHalfFloatScalar" arrow::HalfFloatScalar"(CScalar):
+        npy_half value

Review comment:
       Ah, cool!

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):

Review comment:
       Why do you need to redefine this method?

##########
File path: python/pyarrow/tests/test_scalars.py
##########
@@ -16,427 +16,443 @@
 # under the License.
 
 import datetime
+import decimal
 import pytest
-import unittest
 
 import numpy as np
 
 import pyarrow as pa
 
 
-class TestScalars(unittest.TestCase):
-
-    def test_null_singleton(self):
-        with pytest.raises(Exception):
-            pa.NAType()
+@pytest.mark.parametrize(['value', 'ty', 'klass', 'deprecated'], [
+    (False, None, pa.BooleanScalar, pa.BooleanValue),
+    (True, None, pa.BooleanScalar, pa.BooleanValue),
+    (1, None, pa.Int64Scalar, pa.Int64Value),
+    (-1, None, pa.Int64Scalar, pa.Int64Value),
+    (1, pa.int8(), pa.Int8Scalar, pa.Int8Value),
+    (1, pa.uint8(), pa.UInt8Scalar, pa.UInt8Value),
+    (1, pa.int16(), pa.Int16Scalar, pa.Int16Value),
+    (1, pa.uint16(), pa.UInt16Scalar, pa.UInt16Value),
+    (1, pa.int32(), pa.Int32Scalar, pa.Int32Value),
+    (1, pa.uint32(), pa.UInt32Scalar, pa.UInt32Value),
+    (1, pa.int64(), pa.Int64Scalar, pa.Int64Value),
+    (1, pa.uint64(), pa.UInt64Scalar, pa.UInt64Value),
+    (1.0, None, pa.DoubleScalar, pa.DoubleValue),
+    (np.float16(1.0), pa.float16(), pa.HalfFloatScalar, pa.HalfFloatValue),
+    (1.0, pa.float32(), pa.FloatScalar, pa.FloatValue),
+    ("string", None, pa.StringScalar, pa.StringValue),
+    (b"bytes", None, pa.BinaryScalar, pa.BinaryValue),
+    ([1, 2, 3], None, pa.ListScalar, pa.ListValue),
+    ([1, 2, 3, 4], pa.large_list(pa.int8()), pa.LargeListScalar,
+     pa.LargeListValue),
+    (datetime.date.today(), None, pa.Date32Scalar, pa.Date64Value),
+    (datetime.datetime.now(), None, pa.TimestampScalar, pa.TimestampValue),
+    ({'a': 1, 'b': [1, 2]}, None, pa.StructScalar, pa.StructValue)
+])
+def test_basics(value, ty, klass, deprecated):
+    s = pa.scalar(value, type=ty)
+    assert isinstance(s, klass)
+    assert s == value
+    assert s == s
+    assert s != "else"
+    assert hash(s) == hash(s)

Review comment:
       What's the point of this test? This is going to be trivially true.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())

Review comment:
       `str(buffer, 'utf8')` should work, since `buffer` supports the buffer protocol:
   ```python
   >>> buf = pa.py_buffer(b'abc')                                                                                                                                            
   >>> str(buf, 'utf8')                                                                                                                                                      
   'abc'
   ```

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):
+    pass
 
-cdef class StringValue(ArrayValue):
+
+cdef class ListScalar(Scalar):
     """
-    Concrete class for string (utf8) array elements.
+    Concrete class for list-like scalars.
     """
 
-    def as_py(self):
+    @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 this value as a Python unicode string.
+        Return the number of values.
         """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return len(self.values)
 
-    def as_buffer(self):
+    def __getitem__(self, i):
         """
-        Return a view over this value as a Buffer object.
+        Return the value at the given index.
         """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
+        return self.values[_normalize_index(i, len(self))]
 
-cdef class LargeStringValue(ArrayValue):
-    """
-    Concrete class for large string (utf8) array elements.
-    """
-
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python unicode string.
+        Iterate over this element's values.
         """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return iter(self.values)
 
-    def as_buffer(self):
+    def as_py(self):
         """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python list.
         """
-        cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
+
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class LargeListScalar(ListScalar):
+    pass
+
+
+cdef class StructScalar(Scalar, collections.abc.Mapping):
     """
-    Concrete class for variable-sized binary array elements.
+    Concrete class for struct scalars.
     """
 
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
+    def __len__(self):
+        cdef CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+        return sp.value.size()
+
+    def __iter__(self):
         cdef:
-            const uint8_t* ptr
-            int32_t length
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+            CStructType* dtype = <CStructType*> sp.type.get()
+            vector[shared_ptr[CField]] fields = dtype.fields()
 
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
+        if sp.is_valid:
+            for i in range(dtype.num_fields()):
+                yield frombytes(fields[i].get().name())
 
-    def as_buffer(self):
+    def __contains__(self, key):
+        try:
+            self[key]
+        except IndexError:
+            return False
+        else:
+            return True
+
+    def __getitem__(self, key):
         """
-        Return a view over this value as a Buffer object.
+        Return the child value for the given field.
+
+        Parameters
+        ----------
+        index : Union[int, str]
+            Index / position or name of the field.
+
+        Returns
+        -------
+        result : Scalar
         """
         cdef:
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CFieldRef ref
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        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 LargeBinaryValue(ArrayValue):
-    """
-    Concrete class for large variable-sized binary array elements.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            const uint8_t* ptr
-            int64_t length
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> 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.
-        """
-        cdef:
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
-
-cdef class ListValue(ArrayValue):
-    """
-    Concrete class for list array elements.
-    """
-
-    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)
-
-    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 LargeListValue(ArrayValue):
-    """
-    Concrete class for large list array elements.
-    """
-
-    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 = <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 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())
-
-        return result
-
-
-cdef class MapValue(ArrayValue):
-    """
-    Concrete class for map array elements.
-    """
-
-    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)
+        try:
+            return Scalar.wrap(GetResultValue(sp.field(ref)))
+        except ArrowInvalid:
+            raise IndexError(key)
 
     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 dict.
         """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            key, item = self.getitem(j)
-            result.append((key.as_py(), item.as_py()))
-
-        return result
+        if self.is_valid:
+            return {k: v.as_py() for k, v in self.items()}
+        else:
+            return None
 
 
-cdef class FixedSizeListValue(ArrayValue):
+cdef class MapScalar(ListScalar):
     """
-    Concrete class for fixed size list array elements.
+    Concrete class for map 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()))
+        arr = self.values
+        if arr is None:
+            raise IndexError(i)
+        dct = arr[_normalize_index(i, len(arr))]
+        return (dct['key'], dct['value'])
 
     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)
+        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 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.
-    """
-
-    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)
+        arr = self.values
+        if arr is not None:
+            return list(zip(arr.field('key'), arr.field('value')))
         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.
-
-        The exact type depends on the underlying union member.
-        """
-        return self.getitem(self.index).as_py()
+            return None
 
 
-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 DictionaryScalar(Scalar):
     """
-    Concrete class for struct array elements.
+    Concrete class for dictionary-encoded 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)
+    # @property
+    # def index(self):
+    #     """
+    #     Return this value's underlying index as a scalar.

Review comment:
       Hmm, why is this disabled? Please add a comment.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA

Review comment:
       Hmm, what?

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):

Review comment:
       This `hasattr` check is pointless as, `self` is a `HalfFloatScalar`.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            CDecimal128Scalar* sp = <CDecimal128Scalar*> self.wrapped.get()
+            CDecimal128Type* dtype = <CDecimal128Type*> sp.type.get()
+        if sp.is_valid:
+            return _pydecimal.Decimal(

Review comment:
       Why `_pydecimal`? You should be using the `decimal` module.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):

Review comment:
       Why do you need to redefine this?

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):

Review comment:
       Add docstrings here.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):

Review comment:
       It is not safe to inherit `BinaryScalar.as_buffer`, because it is casting the C++ scalar to `CBinaryScalar`.
   Instead you should redefine that method.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)

Review comment:
       Why do you have to do this twice? If `string_to_tzinfo` is broken, it should be fixed IMO.

##########
File path: python/pyarrow/tests/test_dataset.py
##########
@@ -385,10 +385,12 @@ def test_partitioning():
 
 def test_expression_serialization():
     a = ds.scalar(1)
+    a_ = ds.scalar(pa.scalar(1))

Review comment:
       Uh, can you use another name? Why not `g`?

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):
+    pass
 
-cdef class StringValue(ArrayValue):
+
+cdef class ListScalar(Scalar):
     """
-    Concrete class for string (utf8) array elements.
+    Concrete class for list-like scalars.
     """
 
-    def as_py(self):
+    @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 this value as a Python unicode string.
+        Return the number of values.
         """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return len(self.values)
 
-    def as_buffer(self):
+    def __getitem__(self, i):
         """
-        Return a view over this value as a Buffer object.
+        Return the value at the given index.
         """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
+        return self.values[_normalize_index(i, len(self))]
 
-cdef class LargeStringValue(ArrayValue):
-    """
-    Concrete class for large string (utf8) array elements.
-    """
-
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python unicode string.
+        Iterate over this element's values.
         """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return iter(self.values)
 
-    def as_buffer(self):
+    def as_py(self):
         """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python list.
         """
-        cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
+
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class LargeListScalar(ListScalar):
+    pass
+
+
+cdef class StructScalar(Scalar, collections.abc.Mapping):
     """
-    Concrete class for variable-sized binary array elements.
+    Concrete class for struct scalars.
     """
 
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
+    def __len__(self):
+        cdef CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+        return sp.value.size()
+
+    def __iter__(self):
         cdef:
-            const uint8_t* ptr
-            int32_t length
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+            CStructType* dtype = <CStructType*> sp.type.get()
+            vector[shared_ptr[CField]] fields = dtype.fields()
 
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
+        if sp.is_valid:
+            for i in range(dtype.num_fields()):
+                yield frombytes(fields[i].get().name())
 
-    def as_buffer(self):
+    def __contains__(self, key):
+        try:
+            self[key]
+        except IndexError:
+            return False
+        else:
+            return True
+
+    def __getitem__(self, key):
         """
-        Return a view over this value as a Buffer object.
+        Return the child value for the given field.
+
+        Parameters
+        ----------
+        index : Union[int, str]
+            Index / position or name of the field.
+
+        Returns
+        -------
+        result : Scalar
         """
         cdef:
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CFieldRef ref
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        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 LargeBinaryValue(ArrayValue):
-    """
-    Concrete class for large variable-sized binary array elements.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            const uint8_t* ptr
-            int64_t length
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> 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.
-        """
-        cdef:
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
-
-cdef class ListValue(ArrayValue):
-    """
-    Concrete class for list array elements.
-    """
-
-    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)
-
-    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 LargeListValue(ArrayValue):
-    """
-    Concrete class for large list array elements.
-    """
-
-    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 = <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 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())
-
-        return result
-
-
-cdef class MapValue(ArrayValue):
-    """
-    Concrete class for map array elements.
-    """
-
-    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)
+        try:
+            return Scalar.wrap(GetResultValue(sp.field(ref)))
+        except ArrowInvalid:
+            raise IndexError(key)
 
     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 dict.
         """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            key, item = self.getitem(j)
-            result.append((key.as_py(), item.as_py()))
-
-        return result
+        if self.is_valid:
+            return {k: v.as_py() for k, v in self.items()}
+        else:
+            return None
 
 
-cdef class FixedSizeListValue(ArrayValue):
+cdef class MapScalar(ListScalar):

Review comment:
       Same inheritance problem.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):

Review comment:
       Same here.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -1217,21 +764,50 @@ cdef dict _scalar_classes = {
     _Type_INT16: Int16Scalar,
     _Type_INT32: Int32Scalar,
     _Type_INT64: Int64Scalar,
+    _Type_HALF_FLOAT: HalfFloatScalar,
     _Type_FLOAT: FloatScalar,
     _Type_DOUBLE: DoubleScalar,
+    _Type_DECIMAL: Decimal128Scalar,
+    _Type_DATE32: Date32Scalar,
+    _Type_DATE64: Date64Scalar,
+    _Type_TIME32: Time32Scalar,
+    _Type_TIME64: Time64Scalar,
+    _Type_TIMESTAMP: TimestampScalar,
+    _Type_DURATION: DurationScalar,
+    _Type_BINARY: BinaryScalar,
+    _Type_LARGE_BINARY: LargeBinaryScalar,
+    _Type_FIXED_SIZE_BINARY: FixedSizeBinaryScalar,
     _Type_STRING: StringScalar,
+    _Type_LARGE_STRING: LargeStringScalar,
+    _Type_LIST: ListScalar,
+    _Type_LARGE_LIST: LargeListScalar,
+    _Type_FIXED_SIZE_LIST: FixedSizeListScalar,
+    _Type_STRUCT: StructScalar,
+    _Type_MAP: MapScalar,
+    _Type_DICTIONARY: DictionaryScalar,
+    _Type_SPARSE_UNION: UnionScalar,
+    _Type_DENSE_UNION: UnionScalar,
 }
 
-cdef object box_scalar(DataType type, const shared_ptr[CArray]& sp_array,
-                       int64_t index):
-    cdef ArrayValue value
 
-    if type.type.id() == _Type_NA:
-        return _NULL
-    elif sp_array.get().IsNull(index):
-        return _NULL
-    else:
-        klass = _array_value_classes[type.type.id()]
-        value = klass.__new__(klass)
-        value.init(type, sp_array, index)
-        return value
+def scalar(value, DataType type=None, bint safe=True,

Review comment:
       It seems `safe` isn't used btw.

##########
File path: python/pyarrow/tests/test_scalars.py
##########
@@ -16,427 +16,443 @@
 # under the License.
 
 import datetime
+import decimal
 import pytest
-import unittest
 
 import numpy as np
 
 import pyarrow as pa
 
 
-class TestScalars(unittest.TestCase):
-
-    def test_null_singleton(self):
-        with pytest.raises(Exception):
-            pa.NAType()
+@pytest.mark.parametrize(['value', 'ty', 'klass', 'deprecated'], [
+    (False, None, pa.BooleanScalar, pa.BooleanValue),
+    (True, None, pa.BooleanScalar, pa.BooleanValue),
+    (1, None, pa.Int64Scalar, pa.Int64Value),
+    (-1, None, pa.Int64Scalar, pa.Int64Value),
+    (1, pa.int8(), pa.Int8Scalar, pa.Int8Value),
+    (1, pa.uint8(), pa.UInt8Scalar, pa.UInt8Value),
+    (1, pa.int16(), pa.Int16Scalar, pa.Int16Value),
+    (1, pa.uint16(), pa.UInt16Scalar, pa.UInt16Value),
+    (1, pa.int32(), pa.Int32Scalar, pa.Int32Value),
+    (1, pa.uint32(), pa.UInt32Scalar, pa.UInt32Value),
+    (1, pa.int64(), pa.Int64Scalar, pa.Int64Value),
+    (1, pa.uint64(), pa.UInt64Scalar, pa.UInt64Value),
+    (1.0, None, pa.DoubleScalar, pa.DoubleValue),
+    (np.float16(1.0), pa.float16(), pa.HalfFloatScalar, pa.HalfFloatValue),
+    (1.0, pa.float32(), pa.FloatScalar, pa.FloatValue),
+    ("string", None, pa.StringScalar, pa.StringValue),
+    (b"bytes", None, pa.BinaryScalar, pa.BinaryValue),
+    ([1, 2, 3], None, pa.ListScalar, pa.ListValue),
+    ([1, 2, 3, 4], pa.large_list(pa.int8()), pa.LargeListScalar,
+     pa.LargeListValue),
+    (datetime.date.today(), None, pa.Date32Scalar, pa.Date64Value),
+    (datetime.datetime.now(), None, pa.TimestampScalar, pa.TimestampValue),
+    ({'a': 1, 'b': [1, 2]}, None, pa.StructScalar, pa.StructValue)
+])
+def test_basics(value, ty, klass, deprecated):
+    s = pa.scalar(value, type=ty)
+    assert isinstance(s, klass)
+    assert s == value
+    assert s == s
+    assert s != "else"
+    assert hash(s) == hash(s)
+    assert s.is_valid is True
+    with pytest.warns(FutureWarning):
+        isinstance(s, deprecated)
+
+    s = pa.scalar(None, type=s.type)
+    assert s.is_valid is False
+    assert s.as_py() is None
+
+
+def test_null_singleton():
+    with pytest.raises(Exception):
+        pa.NullScalar()
+
+
+def test_nulls():
+    null = pa.scalar(None)
+    assert null is pa.NA
+    assert null.as_py() is None
+    assert (null == "something") is pa.NA

Review comment:
       Also `assert null == pa.scalar(None)`

##########
File path: python/pyarrow/tests/test_scalars.py
##########
@@ -16,427 +16,443 @@
 # under the License.
 
 import datetime
+import decimal
 import pytest
-import unittest
 
 import numpy as np
 
 import pyarrow as pa
 
 
-class TestScalars(unittest.TestCase):
-
-    def test_null_singleton(self):
-        with pytest.raises(Exception):
-            pa.NAType()
+@pytest.mark.parametrize(['value', 'ty', 'klass', 'deprecated'], [
+    (False, None, pa.BooleanScalar, pa.BooleanValue),
+    (True, None, pa.BooleanScalar, pa.BooleanValue),
+    (1, None, pa.Int64Scalar, pa.Int64Value),
+    (-1, None, pa.Int64Scalar, pa.Int64Value),
+    (1, pa.int8(), pa.Int8Scalar, pa.Int8Value),
+    (1, pa.uint8(), pa.UInt8Scalar, pa.UInt8Value),
+    (1, pa.int16(), pa.Int16Scalar, pa.Int16Value),
+    (1, pa.uint16(), pa.UInt16Scalar, pa.UInt16Value),
+    (1, pa.int32(), pa.Int32Scalar, pa.Int32Value),
+    (1, pa.uint32(), pa.UInt32Scalar, pa.UInt32Value),
+    (1, pa.int64(), pa.Int64Scalar, pa.Int64Value),
+    (1, pa.uint64(), pa.UInt64Scalar, pa.UInt64Value),
+    (1.0, None, pa.DoubleScalar, pa.DoubleValue),
+    (np.float16(1.0), pa.float16(), pa.HalfFloatScalar, pa.HalfFloatValue),
+    (1.0, pa.float32(), pa.FloatScalar, pa.FloatValue),
+    ("string", None, pa.StringScalar, pa.StringValue),
+    (b"bytes", None, pa.BinaryScalar, pa.BinaryValue),
+    ([1, 2, 3], None, pa.ListScalar, pa.ListValue),
+    ([1, 2, 3, 4], pa.large_list(pa.int8()), pa.LargeListScalar,
+     pa.LargeListValue),
+    (datetime.date.today(), None, pa.Date32Scalar, pa.Date64Value),
+    (datetime.datetime.now(), None, pa.TimestampScalar, pa.TimestampValue),
+    ({'a': 1, 'b': [1, 2]}, None, pa.StructScalar, pa.StructValue)
+])
+def test_basics(value, ty, klass, deprecated):
+    s = pa.scalar(value, type=ty)
+    assert isinstance(s, klass)
+    assert s == value
+    assert s == s
+    assert s != "else"
+    assert hash(s) == hash(s)
+    assert s.is_valid is True
+    with pytest.warns(FutureWarning):
+        isinstance(s, deprecated)
+
+    s = pa.scalar(None, type=s.type)
+    assert s.is_valid is False
+    assert s.as_py() is None
+
+
+def test_null_singleton():
+    with pytest.raises(Exception):

Review comment:
       You should a test more precise exception.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -1217,21 +764,50 @@ cdef dict _scalar_classes = {
     _Type_INT16: Int16Scalar,
     _Type_INT32: Int32Scalar,
     _Type_INT64: Int64Scalar,
+    _Type_HALF_FLOAT: HalfFloatScalar,
     _Type_FLOAT: FloatScalar,
     _Type_DOUBLE: DoubleScalar,
+    _Type_DECIMAL: Decimal128Scalar,
+    _Type_DATE32: Date32Scalar,
+    _Type_DATE64: Date64Scalar,
+    _Type_TIME32: Time32Scalar,
+    _Type_TIME64: Time64Scalar,
+    _Type_TIMESTAMP: TimestampScalar,
+    _Type_DURATION: DurationScalar,
+    _Type_BINARY: BinaryScalar,
+    _Type_LARGE_BINARY: LargeBinaryScalar,
+    _Type_FIXED_SIZE_BINARY: FixedSizeBinaryScalar,
     _Type_STRING: StringScalar,
+    _Type_LARGE_STRING: LargeStringScalar,
+    _Type_LIST: ListScalar,
+    _Type_LARGE_LIST: LargeListScalar,
+    _Type_FIXED_SIZE_LIST: FixedSizeListScalar,
+    _Type_STRUCT: StructScalar,
+    _Type_MAP: MapScalar,
+    _Type_DICTIONARY: DictionaryScalar,
+    _Type_SPARSE_UNION: UnionScalar,
+    _Type_DENSE_UNION: UnionScalar,
 }
 
-cdef object box_scalar(DataType type, const shared_ptr[CArray]& sp_array,
-                       int64_t index):
-    cdef ArrayValue value
 
-    if type.type.id() == _Type_NA:
-        return _NULL
-    elif sp_array.get().IsNull(index):
-        return _NULL
-    else:
-        klass = _array_value_classes[type.type.id()]
-        value = klass.__new__(klass)
-        value.init(type, sp_array, index)
-        return value
+def scalar(value, DataType type=None, bint safe=True,
+           MemoryPool memory_pool=None):
+    cdef:
+        PyConversionOptions options
+        shared_ptr[CScalar] scalar
+        shared_ptr[CArray] array
+        shared_ptr[CChunkedArray] chunked
+
+    options.size = 1
+    options.pool = maybe_unbox_memory_pool(memory_pool)
+    # options.from_pandas = from_pandas

Review comment:
       What?

##########
File path: python/pyarrow/tests/test_scalars.py
##########
@@ -16,427 +16,443 @@
 # under the License.
 
 import datetime
+import decimal
 import pytest
-import unittest
 
 import numpy as np
 
 import pyarrow as pa
 
 
-class TestScalars(unittest.TestCase):
-
-    def test_null_singleton(self):
-        with pytest.raises(Exception):
-            pa.NAType()
+@pytest.mark.parametrize(['value', 'ty', 'klass', 'deprecated'], [
+    (False, None, pa.BooleanScalar, pa.BooleanValue),
+    (True, None, pa.BooleanScalar, pa.BooleanValue),
+    (1, None, pa.Int64Scalar, pa.Int64Value),
+    (-1, None, pa.Int64Scalar, pa.Int64Value),
+    (1, pa.int8(), pa.Int8Scalar, pa.Int8Value),
+    (1, pa.uint8(), pa.UInt8Scalar, pa.UInt8Value),
+    (1, pa.int16(), pa.Int16Scalar, pa.Int16Value),
+    (1, pa.uint16(), pa.UInt16Scalar, pa.UInt16Value),
+    (1, pa.int32(), pa.Int32Scalar, pa.Int32Value),
+    (1, pa.uint32(), pa.UInt32Scalar, pa.UInt32Value),
+    (1, pa.int64(), pa.Int64Scalar, pa.Int64Value),
+    (1, pa.uint64(), pa.UInt64Scalar, pa.UInt64Value),
+    (1.0, None, pa.DoubleScalar, pa.DoubleValue),
+    (np.float16(1.0), pa.float16(), pa.HalfFloatScalar, pa.HalfFloatValue),
+    (1.0, pa.float32(), pa.FloatScalar, pa.FloatValue),
+    ("string", None, pa.StringScalar, pa.StringValue),
+    (b"bytes", None, pa.BinaryScalar, pa.BinaryValue),
+    ([1, 2, 3], None, pa.ListScalar, pa.ListValue),
+    ([1, 2, 3, 4], pa.large_list(pa.int8()), pa.LargeListScalar,
+     pa.LargeListValue),
+    (datetime.date.today(), None, pa.Date32Scalar, pa.Date64Value),
+    (datetime.datetime.now(), None, pa.TimestampScalar, pa.TimestampValue),
+    ({'a': 1, 'b': [1, 2]}, None, pa.StructScalar, pa.StructValue)
+])
+def test_basics(value, ty, klass, deprecated):
+    s = pa.scalar(value, type=ty)
+    assert isinstance(s, klass)
+    assert s == value
+    assert s == s
+    assert s != "else"
+    assert hash(s) == hash(s)
+    assert s.is_valid is True
+    with pytest.warns(FutureWarning):
+        isinstance(s, deprecated)
+
+    s = pa.scalar(None, type=s.type)
+    assert s.is_valid is False
+    assert s.as_py() is None
+
+
+def test_null_singleton():
+    with pytest.raises(Exception):
+        pa.NullScalar()
+
+
+def test_nulls():
+    null = pa.scalar(None)
+    assert null is pa.NA
+    assert null.as_py() is None
+    assert (null == "something") is pa.NA

Review comment:
       This looks incorrect and should return `False`.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):

Review comment:
       Same here.
   

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()

Review comment:
       `buffer.to_pybytes()`

##########
File path: python/pyarrow/tests/test_scalars.py
##########
@@ -16,427 +16,443 @@
 # under the License.
 
 import datetime
+import decimal
 import pytest
-import unittest
 
 import numpy as np
 
 import pyarrow as pa
 
 
-class TestScalars(unittest.TestCase):
-
-    def test_null_singleton(self):
-        with pytest.raises(Exception):
-            pa.NAType()
+@pytest.mark.parametrize(['value', 'ty', 'klass', 'deprecated'], [
+    (False, None, pa.BooleanScalar, pa.BooleanValue),
+    (True, None, pa.BooleanScalar, pa.BooleanValue),
+    (1, None, pa.Int64Scalar, pa.Int64Value),
+    (-1, None, pa.Int64Scalar, pa.Int64Value),
+    (1, pa.int8(), pa.Int8Scalar, pa.Int8Value),
+    (1, pa.uint8(), pa.UInt8Scalar, pa.UInt8Value),
+    (1, pa.int16(), pa.Int16Scalar, pa.Int16Value),
+    (1, pa.uint16(), pa.UInt16Scalar, pa.UInt16Value),
+    (1, pa.int32(), pa.Int32Scalar, pa.Int32Value),
+    (1, pa.uint32(), pa.UInt32Scalar, pa.UInt32Value),
+    (1, pa.int64(), pa.Int64Scalar, pa.Int64Value),
+    (1, pa.uint64(), pa.UInt64Scalar, pa.UInt64Value),
+    (1.0, None, pa.DoubleScalar, pa.DoubleValue),
+    (np.float16(1.0), pa.float16(), pa.HalfFloatScalar, pa.HalfFloatValue),
+    (1.0, pa.float32(), pa.FloatScalar, pa.FloatValue),
+    ("string", None, pa.StringScalar, pa.StringValue),
+    (b"bytes", None, pa.BinaryScalar, pa.BinaryValue),
+    ([1, 2, 3], None, pa.ListScalar, pa.ListValue),
+    ([1, 2, 3, 4], pa.large_list(pa.int8()), pa.LargeListScalar,
+     pa.LargeListValue),
+    (datetime.date.today(), None, pa.Date32Scalar, pa.Date64Value),
+    (datetime.datetime.now(), None, pa.TimestampScalar, pa.TimestampValue),
+    ({'a': 1, 'b': [1, 2]}, None, pa.StructScalar, pa.StructValue)
+])
+def test_basics(value, ty, klass, deprecated):
+    s = pa.scalar(value, type=ty)
+    assert isinstance(s, klass)
+    assert s == value
+    assert s == s

Review comment:
       It would be more interesting to instantiate `s` a second time, e.g. `assert s == pa.scalar(value, type=ty)`

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -1217,21 +764,50 @@ cdef dict _scalar_classes = {
     _Type_INT16: Int16Scalar,
     _Type_INT32: Int32Scalar,
     _Type_INT64: Int64Scalar,
+    _Type_HALF_FLOAT: HalfFloatScalar,
     _Type_FLOAT: FloatScalar,
     _Type_DOUBLE: DoubleScalar,
+    _Type_DECIMAL: Decimal128Scalar,
+    _Type_DATE32: Date32Scalar,
+    _Type_DATE64: Date64Scalar,
+    _Type_TIME32: Time32Scalar,
+    _Type_TIME64: Time64Scalar,
+    _Type_TIMESTAMP: TimestampScalar,
+    _Type_DURATION: DurationScalar,
+    _Type_BINARY: BinaryScalar,
+    _Type_LARGE_BINARY: LargeBinaryScalar,
+    _Type_FIXED_SIZE_BINARY: FixedSizeBinaryScalar,
     _Type_STRING: StringScalar,
+    _Type_LARGE_STRING: LargeStringScalar,
+    _Type_LIST: ListScalar,
+    _Type_LARGE_LIST: LargeListScalar,
+    _Type_FIXED_SIZE_LIST: FixedSizeListScalar,
+    _Type_STRUCT: StructScalar,
+    _Type_MAP: MapScalar,
+    _Type_DICTIONARY: DictionaryScalar,
+    _Type_SPARSE_UNION: UnionScalar,
+    _Type_DENSE_UNION: UnionScalar,
 }
 
-cdef object box_scalar(DataType type, const shared_ptr[CArray]& sp_array,
-                       int64_t index):
-    cdef ArrayValue value
 
-    if type.type.id() == _Type_NA:
-        return _NULL
-    elif sp_array.get().IsNull(index):
-        return _NULL
-    else:
-        klass = _array_value_classes[type.type.id()]
-        value = klass.__new__(klass)
-        value.init(type, sp_array, index)
-        return value
+def scalar(value, DataType type=None, bint safe=True,

Review comment:
       Can you add a docstring? Also, make `safe` and `memory_pool` keyword-only.

##########
File path: python/pyarrow/tests/test_scalars.py
##########
@@ -16,427 +16,443 @@
 # under the License.
 
 import datetime
+import decimal
 import pytest
-import unittest
 
 import numpy as np
 
 import pyarrow as pa
 
 
-class TestScalars(unittest.TestCase):
-
-    def test_null_singleton(self):
-        with pytest.raises(Exception):
-            pa.NAType()
+@pytest.mark.parametrize(['value', 'ty', 'klass', 'deprecated'], [
+    (False, None, pa.BooleanScalar, pa.BooleanValue),
+    (True, None, pa.BooleanScalar, pa.BooleanValue),
+    (1, None, pa.Int64Scalar, pa.Int64Value),
+    (-1, None, pa.Int64Scalar, pa.Int64Value),
+    (1, pa.int8(), pa.Int8Scalar, pa.Int8Value),
+    (1, pa.uint8(), pa.UInt8Scalar, pa.UInt8Value),
+    (1, pa.int16(), pa.Int16Scalar, pa.Int16Value),
+    (1, pa.uint16(), pa.UInt16Scalar, pa.UInt16Value),
+    (1, pa.int32(), pa.Int32Scalar, pa.Int32Value),
+    (1, pa.uint32(), pa.UInt32Scalar, pa.UInt32Value),
+    (1, pa.int64(), pa.Int64Scalar, pa.Int64Value),
+    (1, pa.uint64(), pa.UInt64Scalar, pa.UInt64Value),
+    (1.0, None, pa.DoubleScalar, pa.DoubleValue),
+    (np.float16(1.0), pa.float16(), pa.HalfFloatScalar, pa.HalfFloatValue),
+    (1.0, pa.float32(), pa.FloatScalar, pa.FloatValue),
+    ("string", None, pa.StringScalar, pa.StringValue),
+    (b"bytes", None, pa.BinaryScalar, pa.BinaryValue),
+    ([1, 2, 3], None, pa.ListScalar, pa.ListValue),
+    ([1, 2, 3, 4], pa.large_list(pa.int8()), pa.LargeListScalar,
+     pa.LargeListValue),
+    (datetime.date.today(), None, pa.Date32Scalar, pa.Date64Value),
+    (datetime.datetime.now(), None, pa.TimestampScalar, pa.TimestampValue),
+    ({'a': 1, 'b': [1, 2]}, None, pa.StructScalar, pa.StructValue)
+])
+def test_basics(value, ty, klass, deprecated):
+    s = pa.scalar(value, type=ty)
+    assert isinstance(s, klass)
+    assert s == value
+    assert s == s
+    assert s != "else"
+    assert hash(s) == hash(s)
+    assert s.is_valid is True
+    with pytest.warns(FutureWarning):
+        isinstance(s, deprecated)
+
+    s = pa.scalar(None, type=s.type)
+    assert s.is_valid is False
+    assert s.as_py() is None

Review comment:
       Also `assert s != pa.scalar(value, type=ty)`

##########
File path: python/pyarrow/tests/test_parquet.py
##########
@@ -2028,7 +2028,7 @@ def test_filters_invalid_pred_op(tempdir, use_legacy_dataset):
                                     use_legacy_dataset=use_legacy_dataset)
         assert dataset.read().num_rows == 0
 
-    with pytest.raises(ValueError if use_legacy_dataset else TypeError):
+    with pytest.raises(ValueError):
         # dataset API returns TypeError when trying create invalid comparison

Review comment:
       Comment seems obsolete?

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):

Review comment:
       Same here.

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -968,25 +968,31 @@ def test_sequence_timestamp_from_int_with_unit():
     arr_s = pa.array(data, type=s)
     assert len(arr_s) == 1
     assert arr_s.type == s
-    assert repr(arr_s[0]) == "datetime.datetime(1970, 1, 1, 0, 0, 1)"
+    assert repr(arr_s[0].as_py()) == "datetime.datetime(1970, 1, 1, 0, 0, 1)"

Review comment:
       What is being tested here? If you're testing the `as_py()` result, then `repr` isn't needed. Otherwise, you may instead call `str(arr_s[0])`.

##########
File path: python/pyarrow/tests/test_scalars.py
##########
@@ -16,427 +16,443 @@
 # under the License.
 
 import datetime
+import decimal
 import pytest
-import unittest
 
 import numpy as np
 
 import pyarrow as pa
 
 
-class TestScalars(unittest.TestCase):
-
-    def test_null_singleton(self):
-        with pytest.raises(Exception):
-            pa.NAType()
+@pytest.mark.parametrize(['value', 'ty', 'klass', 'deprecated'], [
+    (False, None, pa.BooleanScalar, pa.BooleanValue),
+    (True, None, pa.BooleanScalar, pa.BooleanValue),
+    (1, None, pa.Int64Scalar, pa.Int64Value),
+    (-1, None, pa.Int64Scalar, pa.Int64Value),
+    (1, pa.int8(), pa.Int8Scalar, pa.Int8Value),
+    (1, pa.uint8(), pa.UInt8Scalar, pa.UInt8Value),
+    (1, pa.int16(), pa.Int16Scalar, pa.Int16Value),
+    (1, pa.uint16(), pa.UInt16Scalar, pa.UInt16Value),
+    (1, pa.int32(), pa.Int32Scalar, pa.Int32Value),
+    (1, pa.uint32(), pa.UInt32Scalar, pa.UInt32Value),
+    (1, pa.int64(), pa.Int64Scalar, pa.Int64Value),
+    (1, pa.uint64(), pa.UInt64Scalar, pa.UInt64Value),
+    (1.0, None, pa.DoubleScalar, pa.DoubleValue),
+    (np.float16(1.0), pa.float16(), pa.HalfFloatScalar, pa.HalfFloatValue),
+    (1.0, pa.float32(), pa.FloatScalar, pa.FloatValue),
+    ("string", None, pa.StringScalar, pa.StringValue),
+    (b"bytes", None, pa.BinaryScalar, pa.BinaryValue),
+    ([1, 2, 3], None, pa.ListScalar, pa.ListValue),
+    ([1, 2, 3, 4], pa.large_list(pa.int8()), pa.LargeListScalar,
+     pa.LargeListValue),
+    (datetime.date.today(), None, pa.Date32Scalar, pa.Date64Value),
+    (datetime.datetime.now(), None, pa.TimestampScalar, pa.TimestampValue),
+    ({'a': 1, 'b': [1, 2]}, None, pa.StructScalar, pa.StructValue)
+])
+def test_basics(value, ty, klass, deprecated):
+    s = pa.scalar(value, type=ty)
+    assert isinstance(s, klass)
+    assert s == value
+    assert s == s
+    assert s != "else"
+    assert hash(s) == hash(s)
+    assert s.is_valid is True
+    with pytest.warns(FutureWarning):
+        isinstance(s, deprecated)

Review comment:
       `assert isinstance` perhaps?

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):
+    pass
 
-cdef class StringValue(ArrayValue):
+
+cdef class ListScalar(Scalar):
     """
-    Concrete class for string (utf8) array elements.
+    Concrete class for list-like scalars.
     """
 
-    def as_py(self):
+    @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 this value as a Python unicode string.
+        Return the number of values.
         """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return len(self.values)
 
-    def as_buffer(self):
+    def __getitem__(self, i):
         """
-        Return a view over this value as a Buffer object.
+        Return the value at the given index.
         """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
+        return self.values[_normalize_index(i, len(self))]
 
-cdef class LargeStringValue(ArrayValue):
-    """
-    Concrete class for large string (utf8) array elements.
-    """
-
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python unicode string.
+        Iterate over this element's values.
         """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return iter(self.values)
 
-    def as_buffer(self):
+    def as_py(self):
         """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python list.
         """
-        cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
+
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class LargeListScalar(ListScalar):
+    pass
+
+
+cdef class StructScalar(Scalar, collections.abc.Mapping):
     """
-    Concrete class for variable-sized binary array elements.
+    Concrete class for struct scalars.
     """
 
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
+    def __len__(self):
+        cdef CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+        return sp.value.size()
+
+    def __iter__(self):
         cdef:
-            const uint8_t* ptr
-            int32_t length
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+            CStructType* dtype = <CStructType*> sp.type.get()
+            vector[shared_ptr[CField]] fields = dtype.fields()
 
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
+        if sp.is_valid:
+            for i in range(dtype.num_fields()):
+                yield frombytes(fields[i].get().name())
 
-    def as_buffer(self):
+    def __contains__(self, key):
+        try:
+            self[key]
+        except IndexError:
+            return False
+        else:
+            return True
+
+    def __getitem__(self, key):
         """
-        Return a view over this value as a Buffer object.
+        Return the child value for the given field.
+
+        Parameters
+        ----------
+        index : Union[int, str]
+            Index / position or name of the field.
+
+        Returns
+        -------
+        result : Scalar
         """
         cdef:
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CFieldRef ref
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        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 LargeBinaryValue(ArrayValue):
-    """
-    Concrete class for large variable-sized binary array elements.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            const uint8_t* ptr
-            int64_t length
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> 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.
-        """
-        cdef:
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
-
-cdef class ListValue(ArrayValue):
-    """
-    Concrete class for list array elements.
-    """
-
-    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)
-
-    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 LargeListValue(ArrayValue):
-    """
-    Concrete class for large list array elements.
-    """
-
-    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 = <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 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())
-
-        return result
-
-
-cdef class MapValue(ArrayValue):
-    """
-    Concrete class for map array elements.
-    """
-
-    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)
+        try:
+            return Scalar.wrap(GetResultValue(sp.field(ref)))
+        except ArrowInvalid:
+            raise IndexError(key)

Review comment:
       Ideally, this should be `KeyError` for string keys.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):
+    pass
 
-cdef class StringValue(ArrayValue):
+
+cdef class ListScalar(Scalar):
     """
-    Concrete class for string (utf8) array elements.
+    Concrete class for list-like scalars.
     """
 
-    def as_py(self):
+    @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 this value as a Python unicode string.
+        Return the number of values.
         """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return len(self.values)
 
-    def as_buffer(self):
+    def __getitem__(self, i):
         """
-        Return a view over this value as a Buffer object.
+        Return the value at the given index.
         """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
+        return self.values[_normalize_index(i, len(self))]
 
-cdef class LargeStringValue(ArrayValue):
-    """
-    Concrete class for large string (utf8) array elements.
-    """
-
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python unicode string.
+        Iterate over this element's values.
         """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return iter(self.values)
 
-    def as_buffer(self):
+    def as_py(self):
         """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python list.
         """
-        cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
+
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class LargeListScalar(ListScalar):
+    pass
+
+
+cdef class StructScalar(Scalar, collections.abc.Mapping):
     """
-    Concrete class for variable-sized binary array elements.
+    Concrete class for struct scalars.
     """
 
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
+    def __len__(self):
+        cdef CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+        return sp.value.size()
+
+    def __iter__(self):
         cdef:
-            const uint8_t* ptr
-            int32_t length
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+            CStructType* dtype = <CStructType*> sp.type.get()
+            vector[shared_ptr[CField]] fields = dtype.fields()
 
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
+        if sp.is_valid:
+            for i in range(dtype.num_fields()):
+                yield frombytes(fields[i].get().name())
 
-    def as_buffer(self):
+    def __contains__(self, key):
+        try:
+            self[key]
+        except IndexError:
+            return False
+        else:
+            return True
+
+    def __getitem__(self, key):
         """
-        Return a view over this value as a Buffer object.
+        Return the child value for the given field.
+
+        Parameters
+        ----------
+        index : Union[int, str]
+            Index / position or name of the field.
+
+        Returns
+        -------
+        result : Scalar
         """
         cdef:
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CFieldRef ref
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        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 LargeBinaryValue(ArrayValue):
-    """
-    Concrete class for large variable-sized binary array elements.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            const uint8_t* ptr
-            int64_t length
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> 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.
-        """
-        cdef:
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
-
-cdef class ListValue(ArrayValue):
-    """
-    Concrete class for list array elements.
-    """
-
-    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)
-
-    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 LargeListValue(ArrayValue):
-    """
-    Concrete class for large list array elements.
-    """
-
-    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 = <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 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())
-
-        return result
-
-
-cdef class MapValue(ArrayValue):
-    """
-    Concrete class for map array elements.
-    """
-
-    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)
+        try:
+            return Scalar.wrap(GetResultValue(sp.field(ref)))
+        except ArrowInvalid:
+            raise IndexError(key)
 
     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 dict.
         """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            key, item = self.getitem(j)
-            result.append((key.as_py(), item.as_py()))
-
-        return result
+        if self.is_valid:
+            return {k: v.as_py() for k, v in self.items()}
+        else:
+            return None
 
 
-cdef class FixedSizeListValue(ArrayValue):
+cdef class MapScalar(ListScalar):
     """
-    Concrete class for fixed size list array elements.
+    Concrete class for map 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()))
+        arr = self.values
+        if arr is None:
+            raise IndexError(i)
+        dct = arr[_normalize_index(i, len(arr))]
+        return (dct['key'], dct['value'])
 
     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)
+        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 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.
-    """
-
-    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)
+        arr = self.values
+        if arr is not None:
+            return list(zip(arr.field('key'), arr.field('value')))
         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.
-
-        The exact type depends on the underlying union member.
-        """
-        return self.getitem(self.index).as_py()
+            return None
 
 
-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 DictionaryScalar(Scalar):
     """
-    Concrete class for struct array elements.
+    Concrete class for dictionary-encoded 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)
+    # @property
+    # def index(self):
+    #     """
+    #     Return this value's underlying index as a scalar.
+    #     """
+    #     cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+    #     return Scalar.wrap(sp.index)
 
-        return pyarrow_wrap_array(self.ap.field(index))[self.index]
-
-    def as_py(self):
+    @property
+    def value(self):
         """
-        Return this value as a Python dict.
+        Return this value's underlying dictionary value as a scalar.
         """
-        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.
-    """
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+        return Scalar.wrap(sp.value)
 
     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()
+        value = self.value
+        return None if value is None else 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]
+    # TODO(kszucs): deprecate these
+    # @property
+    # def index_value(self):
+    #     index = self.index
+    #     return None if index is None else 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__()
-
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
-
-    def __eq__(self, other):
-        if hasattr(self, 'as_py'):
-            if isinstance(other, ScalarValue):
-                other = other.as_py()
-            return self.as_py() == other
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bool.
-        """
-        cdef CBooleanScalar* sp = <CBooleanScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt8Scalar(ScalarValue):
-    """
-    Concrete class for uint8 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int8Scalar(ScalarValue):
-    """
-    Concrete class for int8 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt8Scalar* sp = <CInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt16Scalar(ScalarValue):
-    """
-    Concrete class for uint16 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt32Scalar* sp = <CUInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int32Scalar(ScalarValue):
-    """
-    Concrete class for int32 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt32Scalar* sp = <CInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt64Scalar(ScalarValue):
-    """
-    Concrete class for uint64 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt64Scalar* sp = <CUInt64Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int64Scalar(ScalarValue):
-    """
-    Concrete class for int64 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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 self.value
 
 
-cdef class FloatScalar(ScalarValue):
+cdef class UnionScalar(Scalar):
     """
-    Concrete class for float scalars.
+    Concrete class for Union scalars.
     """
 
-    def as_py(self):
+    @property
+    def value(self):
         """
-        Return this value as a Python float.
+        Return this value's underlying dictionary value as a scalar.
         """
-        cdef CFloatScalar* sp = <CFloatScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class DoubleScalar(ScalarValue):
-    """
-    Concrete class for double scalars.
-    """
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()

Review comment:
       UnionScalar

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):
+    pass
 
-cdef class StringValue(ArrayValue):
+
+cdef class ListScalar(Scalar):
     """
-    Concrete class for string (utf8) array elements.
+    Concrete class for list-like scalars.
     """
 
-    def as_py(self):
+    @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 this value as a Python unicode string.
+        Return the number of values.
         """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return len(self.values)
 
-    def as_buffer(self):
+    def __getitem__(self, i):
         """
-        Return a view over this value as a Buffer object.
+        Return the value at the given index.
         """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
+        return self.values[_normalize_index(i, len(self))]
 
-cdef class LargeStringValue(ArrayValue):
-    """
-    Concrete class for large string (utf8) array elements.
-    """
-
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python unicode string.
+        Iterate over this element's values.
         """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return iter(self.values)
 
-    def as_buffer(self):
+    def as_py(self):
         """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python list.
         """
-        cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
+
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class LargeListScalar(ListScalar):

Review comment:
       Same here.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):
+    pass
 
-cdef class StringValue(ArrayValue):
+
+cdef class ListScalar(Scalar):
     """
-    Concrete class for string (utf8) array elements.
+    Concrete class for list-like scalars.
     """
 
-    def as_py(self):
+    @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 this value as a Python unicode string.
+        Return the number of values.
         """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return len(self.values)
 
-    def as_buffer(self):
+    def __getitem__(self, i):
         """
-        Return a view over this value as a Buffer object.
+        Return the value at the given index.
         """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
+        return self.values[_normalize_index(i, len(self))]
 
-cdef class LargeStringValue(ArrayValue):
-    """
-    Concrete class for large string (utf8) array elements.
-    """
-
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python unicode string.
+        Iterate over this element's values.
         """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return iter(self.values)
 
-    def as_buffer(self):
+    def as_py(self):
         """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python list.
         """
-        cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
+
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+cdef class FixedSizeListScalar(ListScalar):

Review comment:
       Like above, you cannot inherit `ListScalar.values` safely.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):
+    pass
 
-cdef class StringValue(ArrayValue):
+
+cdef class ListScalar(Scalar):
     """
-    Concrete class for string (utf8) array elements.
+    Concrete class for list-like scalars.
     """
 
-    def as_py(self):
+    @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 this value as a Python unicode string.
+        Return the number of values.
         """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return len(self.values)
 
-    def as_buffer(self):
+    def __getitem__(self, i):
         """
-        Return a view over this value as a Buffer object.
+        Return the value at the given index.
         """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
+        return self.values[_normalize_index(i, len(self))]
 
-cdef class LargeStringValue(ArrayValue):
-    """
-    Concrete class for large string (utf8) array elements.
-    """
-
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python unicode string.
+        Iterate over this element's values.
         """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return iter(self.values)
 
-    def as_buffer(self):
+    def as_py(self):
         """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python list.
         """
-        cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
+
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class LargeListScalar(ListScalar):
+    pass
+
+
+cdef class StructScalar(Scalar, collections.abc.Mapping):
     """
-    Concrete class for variable-sized binary array elements.
+    Concrete class for struct scalars.
     """
 
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
+    def __len__(self):
+        cdef CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+        return sp.value.size()
+
+    def __iter__(self):
         cdef:
-            const uint8_t* ptr
-            int32_t length
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+            CStructType* dtype = <CStructType*> sp.type.get()
+            vector[shared_ptr[CField]] fields = dtype.fields()
 
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
+        if sp.is_valid:
+            for i in range(dtype.num_fields()):
+                yield frombytes(fields[i].get().name())
 
-    def as_buffer(self):
+    def __contains__(self, key):
+        try:
+            self[key]
+        except IndexError:
+            return False
+        else:
+            return True
+
+    def __getitem__(self, key):
         """
-        Return a view over this value as a Buffer object.
+        Return the child value for the given field.
+
+        Parameters
+        ----------
+        index : Union[int, str]
+            Index / position or name of the field.
+
+        Returns
+        -------
+        result : Scalar
         """
         cdef:
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CFieldRef ref
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        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 LargeBinaryValue(ArrayValue):
-    """
-    Concrete class for large variable-sized binary array elements.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            const uint8_t* ptr
-            int64_t length
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> 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.
-        """
-        cdef:
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
-
-cdef class ListValue(ArrayValue):
-    """
-    Concrete class for list array elements.
-    """
-
-    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)
-
-    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 LargeListValue(ArrayValue):
-    """
-    Concrete class for large list array elements.
-    """
-
-    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 = <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 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())
-
-        return result
-
-
-cdef class MapValue(ArrayValue):
-    """
-    Concrete class for map array elements.
-    """
-
-    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)
+        try:
+            return Scalar.wrap(GetResultValue(sp.field(ref)))
+        except ArrowInvalid:
+            raise IndexError(key)
 
     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 dict.
         """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            key, item = self.getitem(j)
-            result.append((key.as_py(), item.as_py()))
-
-        return result
+        if self.is_valid:
+            return {k: v.as_py() for k, v in self.items()}
+        else:
+            return None
 
 
-cdef class FixedSizeListValue(ArrayValue):
+cdef class MapScalar(ListScalar):
     """
-    Concrete class for fixed size list array elements.
+    Concrete class for map 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()))
+        arr = self.values
+        if arr is None:
+            raise IndexError(i)
+        dct = arr[_normalize_index(i, len(arr))]
+        return (dct['key'], dct['value'])
 
     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)
+        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 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.
-    """
-
-    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)
+        arr = self.values
+        if arr is not None:
+            return list(zip(arr.field('key'), arr.field('value')))
         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.
-
-        The exact type depends on the underlying union member.
-        """
-        return self.getitem(self.index).as_py()
+            return None
 
 
-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 DictionaryScalar(Scalar):
     """
-    Concrete class for struct array elements.
+    Concrete class for dictionary-encoded 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)
+    # @property
+    # def index(self):
+    #     """
+    #     Return this value's underlying index as a scalar.
+    #     """
+    #     cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+    #     return Scalar.wrap(sp.index)
 
-        return pyarrow_wrap_array(self.ap.field(index))[self.index]
-
-    def as_py(self):
+    @property
+    def value(self):
         """
-        Return this value as a Python dict.
+        Return this value's underlying dictionary value as a scalar.
         """
-        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.
-    """
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+        return Scalar.wrap(sp.value)
 
     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()
+        value = self.value
+        return None if value is None else 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]
+    # TODO(kszucs): deprecate these
+    # @property
+    # def index_value(self):
+    #     index = self.index
+    #     return None if index is None else 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__()
-
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
-
-    def __eq__(self, other):
-        if hasattr(self, 'as_py'):
-            if isinstance(other, ScalarValue):
-                other = other.as_py()
-            return self.as_py() == other
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bool.
-        """
-        cdef CBooleanScalar* sp = <CBooleanScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt8Scalar(ScalarValue):
-    """
-    Concrete class for uint8 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int8Scalar(ScalarValue):
-    """
-    Concrete class for int8 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt8Scalar* sp = <CInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt16Scalar(ScalarValue):
-    """
-    Concrete class for uint16 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt32Scalar* sp = <CUInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int32Scalar(ScalarValue):
-    """
-    Concrete class for int32 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt32Scalar* sp = <CInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt64Scalar(ScalarValue):
-    """
-    Concrete class for uint64 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt64Scalar* sp = <CUInt64Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int64Scalar(ScalarValue):
-    """
-    Concrete class for int64 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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 self.value
 
 
-cdef class FloatScalar(ScalarValue):
+cdef class UnionScalar(Scalar):
     """
-    Concrete class for float scalars.
+    Concrete class for Union scalars.
     """
 
-    def as_py(self):
+    @property
+    def value(self):
         """
-        Return this value as a Python float.
+        Return this value's underlying dictionary value as a scalar.

Review comment:
       There's no dictionary here.

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -16,1198 +16,745 @@
 # under the License.
 
 
-_NULL = NA = None
+import collections
 
 
 cdef class Scalar:
     """
-    The base class for all array elements.
+    The base class for scalars.
     """
 
+    def __init__(self):
+        raise TypeError("Do not call {}'s constructor directly, use "
+                        "pa.scalar() instead.".format(self.__class__.__name__))
 
-cdef class NullType(Scalar):
-    """
-    Singleton for null array elements.
-    """
-    # TODO rename this NullValue?
+    cdef void init(self, const shared_ptr[CScalar]& wrapped):
+        self.wrapped = wrapped
 
-    def __cinit__(self):
-        global NA
-        if NA is not None:
-            raise Exception('Cannot create multiple NAType instances')
+    @staticmethod
+    cdef wrap(const shared_ptr[CScalar]& wrapped):
+        cdef:
+            Scalar self
+            Type type_id = wrapped.get().type.get().id()
+
+        if type_id == _Type_NA:
+            return _NULL
+
+        typ = _scalar_classes[type_id]
+        self = typ.__new__(typ)
+        self.init(wrapped)
+
+        return self
+
+    cdef inline shared_ptr[CScalar] unwrap(self) nogil:
+        return self.wrapped
+
+    @property
+    def type(self):
+        return pyarrow_wrap_data_type(self.wrapped.get().type)
 
-        self.type = null()
+    @property
+    def is_valid(self):
+        return self.wrapped.get().is_valid
 
     def __repr__(self):
-        return 'NULL'
+        return '<pyarrow.{}: {!r}>'.format(
+            self.__class__.__name__, self.as_py()
+        )
 
-    def as_py(self):
-        """
-        Return None
-        """
-        return None
+    def __str__(self):
+        return str(self.as_py())
+
+    def equals(self, Scalar other):
+        return self.wrapped.get().Equals(other.unwrap().get()[0])
 
     def __eq__(self, other):
-        return NA
+        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):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def as_py(self):
+        raise NotImplementedError()
 
 
-_NULL = NA = NullType()
+_NULL = NA = None
 
 
-cdef class ArrayValue(Scalar):
+cdef class NullScalar(Scalar):
     """
-    The base class for non-null array elements.
+    Concrete class for null scalars.
     """
 
-    def __init__(self):
-        raise TypeError("Do not call {}'s constructor directly, use array "
-                        "subscription instead."
-                        .format(self.__class__.__name__))
+    def __cinit__(self):
+        global NA
+        if NA is not None:
+            raise Exception('Cannot create multiple NAType instances')
+        self.init(shared_ptr[CScalar](new CNullScalar()))
 
-    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)
+    def __init__(self):
+        pass
 
-    cdef void _set_array(self, const shared_ptr[CArray]& sp_array):
-        self.sp_array = sp_array
+    def __eq__(self, other):
+        return NA
 
-    def __repr__(self):
-        if hasattr(self, 'as_py'):
-            return repr(self.as_py())
-        else:
-            return super(Scalar, self).__repr__()
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
 
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
+    def as_py(self):
+        """
+        Return this value as a Python None.
+        """
+        return None
 
-    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()")
 
-    def __hash__(self):
-        return hash(self.as_py())
+_NULL = NA = NullScalar()
 
 
-cdef class BooleanValue(ArrayValue):
+cdef class BooleanScalar(Scalar):
     """
-    Concrete class for boolean array elements.
+    Concrete class for boolean scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python bool.
         """
-        cdef CBooleanArray* ap = <CBooleanArray*> 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 Int8Value(ArrayValue):
+cdef class UInt8Scalar(Scalar):
     """
-    Concrete class for int8 array elements.
+    Concrete class for uint8 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)
+        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt8Value(ArrayValue):
+cdef class Int8Scalar(Scalar):
     """
-    Concrete class for uint8 array elements.
+    Concrete class for int8 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt8Array* ap = <CUInt8Array*> 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 Int16Value(ArrayValue):
+cdef class UInt16Scalar(Scalar):
     """
-    Concrete class for int16 array elements.
+    Concrete class for uint16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CInt16Array* ap = <CInt16Array*> 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 UInt16Value(ArrayValue):
+cdef class Int16Scalar(Scalar):
     """
-    Concrete class for uint16 array elements.
+    Concrete class for int16 scalars.
     """
 
     def as_py(self):
         """
         Return this value as a Python int.
         """
-        cdef CUInt16Array* ap = <CUInt16Array*> 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 Int32Value(ArrayValue):
+cdef class UInt32Scalar(Scalar):
     """
-    Concrete class for int32 array elements.
+    Concrete class for uint32 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 CUInt32Scalar* sp = <CUInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt32Value(ArrayValue):
+cdef class Int32Scalar(Scalar):
     """
-    Concrete class for uint32 array elements.
+    Concrete class for int32 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 CInt32Scalar* sp = <CInt32Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Int64Value(ArrayValue):
+cdef class UInt64Scalar(Scalar):
     """
-    Concrete class for int64 array elements.
+    Concrete class for uint64 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 CUInt64Scalar* sp = <CUInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class UInt64Value(ArrayValue):
+cdef class Int64Scalar(Scalar):
     """
-    Concrete class for uint64 array elements.
+    Concrete class for int64 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 CInt64Scalar* sp = <CInt64Scalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date32Value(ArrayValue):
+cdef class HalfFloatScalar(Scalar):
     """
-    Concrete class for date32 array elements.
+    Concrete class for float scalars.
     """
 
+    def __hash__(self):
+        cdef CScalarHash hasher
+        return hasher(self.wrapped)
+
+    def __eq__(self, other):
+        if hasattr(self, 'as_py'):
+            if isinstance(other, Scalar):
+                other = other.as_py()
+            return self.as_py() == other
+        else:
+            raise NotImplementedError
+
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate32Array* ap = <CDate32Array*> self.sp_array.get()
+        cdef CHalfFloatScalar* sp = <CHalfFloatScalar*> self.wrapped.get()
+        return PyHalf_FromHalf(sp.value) if sp.is_valid else None
 
-        # Shift to seconds since epoch
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(days=ap.Value(self.index)))
+
+cdef class FloatScalar(Scalar):
+    """
+    Concrete class for float scalars.
+    """
+
+    def as_py(self):
+        """
+        Return this value as a Python float.
+        """
+        cdef CFloatScalar* sp = <CFloatScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Date64Value(ArrayValue):
+cdef class DoubleScalar(Scalar):
     """
-    Concrete class for date64 array elements.
+    Concrete class for double scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.datetime instance.
+        Return this value as a Python float.
         """
-        cdef CDate64Array* ap = <CDate64Array*> self.sp_array.get()
-        return (datetime.date(1970, 1, 1) +
-                datetime.timedelta(
-                    days=ap.Value(self.index) / 86400000))
+        cdef CDoubleScalar* sp = <CDoubleScalar*> self.wrapped.get()
+        return sp.value if sp.is_valid else None
 
 
-cdef class Time32Value(ArrayValue):
+cdef class Decimal128Scalar(Scalar):
     """
-    Concrete class for time32 array elements.
+    Concrete class for decimal128 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python Decimal.
         """
         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()
+            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 _box_time_milli(ap.Value(self.index))
+            return None
 
 
-cdef class Time64Value(ArrayValue):
+cdef class Date32Scalar(Scalar):
     """
-    Concrete class for time64 array elements.
+    Concrete class for date32 scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python datetime.timedelta instance.
+        Return this value as a Python datetime.datetime instance.
         """
-        cdef:
-            CTime64Array* ap = <CTime64Array*> self.sp_array.get()
-            CTime64Type* dtype = <CTime64Type*> ap.type().get()
+        cdef CDate32Scalar* sp = <CDate32Scalar*> self.wrapped.get()
 
-        cdef int64_t val = ap.Value(self.index)
-        if dtype.unit() == TimeUnit_MICRO:
-            return _box_time_micro(val)
+        if sp.is_valid:
+            # shift to seconds since epoch
+            return (
+                datetime.date(1970, 1, 1) + datetime.timedelta(days=sp.value)
+            )
         else:
-            return (datetime.datetime(1970, 1, 1) +
-                    datetime.timedelta(microseconds=val / 1000)).time()
-
+            return None
 
-cpdef _box_time_milli(int64_t val):
-    delta = datetime.timedelta(milliseconds=val)
-    return (datetime.datetime(1970, 1, 1) + delta).time()
 
+cdef class Date64Scalar(Scalar):
+    """
+    Concrete class for date64 scalars.
+    """
 
-cpdef _box_time_micro(int64_t val):
-    return (datetime.datetime(1970, 1, 1) +
-            datetime.timedelta(microseconds=val)).time()
+    def as_py(self):
+        """
+        Return this value as a Python datetime.datetime instance.
+        """
+        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 dict _DATETIME_CONVERSION_FUNCTIONS = {}
-cdef c_bool _datetime_conversion_initialized = False
 
+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)
 
-cdef _add_micros_maybe_localize(dt, micros, tzinfo):
-    import pytz
-    dt = dt.replace(microsecond=micros)
+    dt = datetime.datetime(1970, 1, 1) + delta
+    # adjust timezone if set to the datatype
     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)
 
+    return dt
 
-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)
 
+cdef class Time32Scalar(Scalar):
+    """
+    Concrete class for time32 scalars.
+    """
 
-def _seconds_to_datetime(v, tzinfo):
-    dt = _datetime_from_seconds(v)
-    return _add_micros_maybe_localize(dt, 0, tzinfo)
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime32Scalar* sp = <CTime32Scalar*> self.wrapped.get()
+            CTime32Type* dtype = <CTime32Type*> sp.type.get()
 
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
-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
-    })
+cdef class Time64Scalar(Scalar):
+    """
+    Concrete class for time64 scalars.
+    """
 
-    try:
-        import pandas as pd
-        _DATETIME_CONVERSION_FUNCTIONS[TimeUnit_NANO] = (
-            lambda x, tzinfo: pd.Timestamp(
-                x, tz=tzinfo, unit='ns',
-            )
-        )
-    except ImportError:
-        pass
+    def as_py(self):
+        """
+        Return this value as a Python datetime.timedelta instance.
+        """
+        cdef:
+            CTime64Scalar* sp = <CTime64Scalar*> self.wrapped.get()
+            CTime64Type* dtype = <CTime64Type*> sp.type.get()
 
-    _datetime_conversion_initialized = True
-    return _DATETIME_CONVERSION_FUNCTIONS
+        if sp.is_valid:
+            return _datetime_from_int(sp.value, unit=dtype.unit()).time()
+        else:
+            return None
 
 
-cdef class TimestampValue(ArrayValue):
+cdef class TimestampScalar(Scalar):
     """
-    Concrete class for timestamp array elements.
+    Concrete class for timestamp 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)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CTimestampArray* ap = <CTimestampArray*> self.sp_array.get()
-        cdef CTimestampType* dtype = <CTimestampType*> ap.type().get()
+        cdef:
+            CTimestampScalar* sp = <CTimestampScalar*> self.wrapped.get()
+            CTimestampType* dtype = <CTimestampType*> sp.type.get()
 
-        value = self.value
+        if not sp.is_valid:
+            return None
 
         if not dtype.timezone().empty():
             tzinfo = string_to_tzinfo(frombytes(dtype.timezone()))
+            if not isinstance(tzinfo, datetime.tzinfo):
+                tzinfo = string_to_tzinfo(tzinfo)
         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)
-
-
-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 _datetime_from_int(sp.value, unit=dtype.unit(), tzinfo=tzinfo)
 
-    return _TIMEDELTA_CONVERSION_FUNCTIONS
 
-
-cdef class DurationValue(ArrayValue):
+cdef class DurationScalar(Scalar):
     """
-    Concrete class for duration array elements.
+    Concrete class for duration scalars.
     """
 
     @property
     def value(self):
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        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 Pandas Timestamp instance (if available),
         otherwise as a Python datetime.timedelta instance.
         """
-        cdef CDurationArray* ap = <CDurationArray*> self.sp_array.get()
-        cdef CDurationType* dtype = <CDurationType*> ap.type().get()
+        cdef:
+            CDurationScalar* sp = <CDurationScalar*> self.wrapped.get()
+            CDurationType* dtype = <CDurationType*> sp.type.get()
+            TimeUnit unit = dtype.unit()
+
+        if not sp.is_valid:
+            return None
 
-        cdef int64_t value = ap.Value(self.index)
-        converter = _timedelta_conversion_functions()[dtype.unit()]
-        return converter(value)
+        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 HalfFloatValue(ArrayValue):
+cdef class BinaryScalar(Scalar):
     """
-    Concrete class for float16 array elements.
+    Concrete class for binary-like scalars.
     """
 
-    def as_py(self):
+    def as_buffer(self):
         """
-        Return this value as a Python float.
+        Return a view over this value as a Buffer object.
         """
-        cdef CHalfFloatArray* ap = <CHalfFloatArray*> self.sp_array.get()
-        return PyHalf_FromHalf(ap.Value(self.index))
-
-
-cdef class FloatValue(ArrayValue):
-    """
-    Concrete class for float32 array elements.
-    """
+        cdef CBinaryScalar* sp = <CBinaryScalar*> self.wrapped.get()
+        return pyarrow_wrap_buffer(sp.value) if sp.is_valid else None
 
     def as_py(self):
         """
-        Return this value as a Python float.
+        Return this value as a Python bytes.
         """
-        cdef CFloatArray* ap = <CFloatArray*> self.sp_array.get()
-        return ap.Value(self.index)
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return self.as_buffer().to_pybytes()
+        else:
+            return None
 
 
-cdef class DoubleValue(ArrayValue):
-    """
-    Concrete class for float64 array elements.
-    """
+cdef class LargeBinaryScalar(BinaryScalar):
+    pass
 
-    def as_py(self):
-        """
-        Return this value as a Python float.
-        """
-        cdef CDoubleArray* ap = <CDoubleArray*> self.sp_array.get()
-        return ap.Value(self.index)
+
+cdef class FixedSizeBinaryScalar(BinaryScalar):
+    pass
 
 
-cdef class DecimalValue(ArrayValue):
+cdef class StringScalar(BinaryScalar):
     """
-    Concrete class for decimal128 array elements.
+    Concrete class for string-like (utf8) scalars.
     """
 
     def as_py(self):
         """
-        Return this value as a Python Decimal.
+        Return this value as a Python string.
         """
-        cdef:
-            CDecimal128Array* ap = <CDecimal128Array*> self.sp_array.get()
-            c_string s = ap.FormatValue(self.index)
-        return _pydecimal.Decimal(s.decode('utf8'))
+        buffer = self.as_buffer()
+        if buffer is not None:
+            return frombytes(self.as_buffer().to_pybytes())
+        else:
+            return None
+
 
+cdef class LargeStringScalar(StringScalar):
+    pass
 
-cdef class StringValue(ArrayValue):
+
+cdef class ListScalar(Scalar):
     """
-    Concrete class for string (utf8) array elements.
+    Concrete class for list-like scalars.
     """
 
-    def as_py(self):
+    @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 this value as a Python unicode string.
+        Return the number of values.
         """
-        cdef CStringArray* ap = <CStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return len(self.values)
 
-    def as_buffer(self):
+    def __getitem__(self, i):
         """
-        Return a view over this value as a Buffer object.
+        Return the value at the given index.
         """
-        cdef:
-            CStringArray* ap = <CStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
+        return self.values[_normalize_index(i, len(self))]
 
-cdef class LargeStringValue(ArrayValue):
-    """
-    Concrete class for large string (utf8) array elements.
-    """
-
-    def as_py(self):
+    def __iter__(self):
         """
-        Return this value as a Python unicode string.
+        Iterate over this element's values.
         """
-        cdef CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-        return ap.GetString(self.index).decode('utf-8')
+        return iter(self.values)
 
-    def as_buffer(self):
+    def as_py(self):
         """
-        Return a view over this value as a Buffer object.
+        Return this value as a Python list.
         """
-        cdef:
-            CLargeStringArray* ap = <CLargeStringArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+        arr = self.values
+        return None if arr is None else arr.to_pylist()
+
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+cdef class FixedSizeListScalar(ListScalar):
+    pass
 
 
-cdef class BinaryValue(ArrayValue):
+cdef class LargeListScalar(ListScalar):
+    pass
+
+
+cdef class StructScalar(Scalar, collections.abc.Mapping):
     """
-    Concrete class for variable-sized binary array elements.
+    Concrete class for struct scalars.
     """
 
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
+    def __len__(self):
+        cdef CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+        return sp.value.size()
+
+    def __iter__(self):
         cdef:
-            const uint8_t* ptr
-            int32_t length
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
+            CStructType* dtype = <CStructType*> sp.type.get()
+            vector[shared_ptr[CField]] fields = dtype.fields()
 
-        ptr = ap.GetValue(self.index, &length)
-        return cp.PyBytes_FromStringAndSize(<const char*>(ptr), length)
+        if sp.is_valid:
+            for i in range(dtype.num_fields()):
+                yield frombytes(fields[i].get().name())
 
-    def as_buffer(self):
+    def __contains__(self, key):
+        try:
+            self[key]
+        except IndexError:
+            return False
+        else:
+            return True
+
+    def __getitem__(self, key):
         """
-        Return a view over this value as a Buffer object.
+        Return the child value for the given field.
+
+        Parameters
+        ----------
+        index : Union[int, str]
+            Index / position or name of the field.
+
+        Returns
+        -------
+        result : Scalar
         """
         cdef:
-            CBinaryArray* ap = <CBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
+            CFieldRef ref
+            CStructScalar* sp = <CStructScalar*> self.wrapped.get()
 
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
+        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 LargeBinaryValue(ArrayValue):
-    """
-    Concrete class for large variable-sized binary array elements.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bytes object.
-        """
-        cdef:
-            const uint8_t* ptr
-            int64_t length
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> 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.
-        """
-        cdef:
-            CLargeBinaryArray* ap = <CLargeBinaryArray*> self.sp_array.get()
-            shared_ptr[CBuffer] buf
-
-        buf = SliceBuffer(ap.value_data(), ap.value_offset(self.index),
-                          ap.value_length(self.index))
-        return pyarrow_wrap_buffer(buf)
-
-
-cdef class ListValue(ArrayValue):
-    """
-    Concrete class for list array elements.
-    """
-
-    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)
-
-    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 LargeListValue(ArrayValue):
-    """
-    Concrete class for large list array elements.
-    """
-
-    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 = <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 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())
-
-        return result
-
-
-cdef class MapValue(ArrayValue):
-    """
-    Concrete class for map array elements.
-    """
-
-    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)
+        try:
+            return Scalar.wrap(GetResultValue(sp.field(ref)))
+        except ArrowInvalid:
+            raise IndexError(key)
 
     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 dict.
         """
-        cdef:
-            int64_t j
-            list result = []
-
-        for j in range(len(self)):
-            key, item = self.getitem(j)
-            result.append((key.as_py(), item.as_py()))
-
-        return result
+        if self.is_valid:
+            return {k: v.as_py() for k, v in self.items()}
+        else:
+            return None
 
 
-cdef class FixedSizeListValue(ArrayValue):
+cdef class MapScalar(ListScalar):
     """
-    Concrete class for fixed size list array elements.
+    Concrete class for map 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()))
+        arr = self.values
+        if arr is None:
+            raise IndexError(i)
+        dct = arr[_normalize_index(i, len(arr))]
+        return (dct['key'], dct['value'])
 
     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)
+        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 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.
-    """
-
-    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)
+        arr = self.values
+        if arr is not None:
+            return list(zip(arr.field('key'), arr.field('value')))
         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.
-
-        The exact type depends on the underlying union member.
-        """
-        return self.getitem(self.index).as_py()
+            return None
 
 
-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 DictionaryScalar(Scalar):
     """
-    Concrete class for struct array elements.
+    Concrete class for dictionary-encoded 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)
+    # @property
+    # def index(self):
+    #     """
+    #     Return this value's underlying index as a scalar.
+    #     """
+    #     cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+    #     return Scalar.wrap(sp.index)
 
-        return pyarrow_wrap_array(self.ap.field(index))[self.index]
-
-    def as_py(self):
+    @property
+    def value(self):
         """
-        Return this value as a Python dict.
+        Return this value's underlying dictionary value as a scalar.
         """
-        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.
-    """
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+        return Scalar.wrap(sp.value)
 
     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()
+        value = self.value
+        return None if value is None else 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]
+    # TODO(kszucs): deprecate these
+    # @property
+    # def index_value(self):
+    #     index = self.index
+    #     return None if index is None else 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__()
-
-    def __str__(self):
-        if hasattr(self, 'as_py'):
-            return str(self.as_py())
-        else:
-            return super(Scalar, self).__str__()
-
-    def __eq__(self, other):
-        if hasattr(self, 'as_py'):
-            if isinstance(other, ScalarValue):
-                other = other.as_py()
-            return self.as_py() == other
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python bool.
-        """
-        cdef CBooleanScalar* sp = <CBooleanScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt8Scalar(ScalarValue):
-    """
-    Concrete class for uint8 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt8Scalar* sp = <CUInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int8Scalar(ScalarValue):
-    """
-    Concrete class for int8 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt8Scalar* sp = <CInt8Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt16Scalar(ScalarValue):
-    """
-    Concrete class for uint16 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt32Scalar* sp = <CUInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int32Scalar(ScalarValue):
-    """
-    Concrete class for int32 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CInt32Scalar* sp = <CInt32Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class UInt64Scalar(ScalarValue):
-    """
-    Concrete class for uint64 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        cdef CUInt64Scalar* sp = <CUInt64Scalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class Int64Scalar(ScalarValue):
-    """
-    Concrete class for int64 scalars.
-    """
-
-    def as_py(self):
-        """
-        Return this value as a Python int.
-        """
-        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 self.value
 
 
-cdef class FloatScalar(ScalarValue):
+cdef class UnionScalar(Scalar):
     """
-    Concrete class for float scalars.
+    Concrete class for Union scalars.
     """
 
-    def as_py(self):
+    @property
+    def value(self):
         """
-        Return this value as a Python float.
+        Return this value's underlying dictionary value as a scalar.
         """
-        cdef CFloatScalar* sp = <CFloatScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class DoubleScalar(ScalarValue):
-    """
-    Concrete class for double scalars.
-    """
+        cdef CDictionaryScalar* sp = <CDictionaryScalar*> self.wrapped.get()
+        return Scalar.wrap(sp.value)
 
     def as_py(self):
         """
-        Return this value as a Python float.
-        """
-        cdef CDoubleScalar* sp = <CDoubleScalar*> self.sp_scalar.get()
-        return sp.value if sp.is_valid else None
-
-
-cdef class StringScalar(ScalarValue):
-    """
-    Concrete class for string scalars.
-    """
+        Return this value as a Python object.
 
-    def as_py(self):
-        """
-        Return this value as a Python string.
+        The exact type depends on the dictionary value type.

Review comment:
       Not "dictionary"




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