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/12/17 19:38:11 UTC

[GitHub] [arrow] pitrou commented on a change in pull request #8894: ARROW-10322: [C++][Dataset] Minimize Expression

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



##########
File path: cpp/src/arrow/array/array_struct_test.cc
##########
@@ -582,4 +583,39 @@ TEST_F(TestStructBuilder, TestSlice) {
   ASSERT_EQ(list_field->null_count(), 1);
 }
 
+TEST(TestFieldRef, GetChildren) {
+  auto struct_array = ArrayFromJSON(struct_({field("a", float64())}), R"([
+    {"a": 6.125},
+    {"a": 0.0},
+    {"a": -1}
+  ])");
+
+  ASSERT_OK_AND_ASSIGN(auto a, FieldRef("a").GetOne(*struct_array));
+  auto expected_a = ArrayFromJSON(float64(), "[6.125, 0.0, -1]");
+  AssertArraysEqual(*a, *expected_a);
+
+  auto ToChunked = [struct_array](int64_t midpoint) {
+    return ChunkedArray(
+        ArrayVector{
+            struct_array->Slice(0, midpoint),
+            struct_array->Slice(midpoint),
+        },
+        struct_array->type());
+  };
+  AssertChunkedEquivalent(ToChunked(1), ToChunked(2));

Review comment:
       It's not clear to me why this test is here. Was there a particular slicing bug with `FieldRef::GetOne`?

##########
File path: cpp/examples/arrow/dataset-parquet-scan-example.cc
##########
@@ -62,7 +60,8 @@ struct Configuration {
 
   // Indicates the filter by which rows will be filtered. This optimization can
   // make use of partition information and/or file metadata if possible.
-  std::shared_ptr<ds::Expression> filter = ("total_amount"_ > 1000.0f).Copy();
+  ds::Expression filter =
+      ds::greater(ds::field_ref("total_amount"), ds::literal(1000.0f));

Review comment:
       Are we going to bring back the syntactic sugar operators or is it just not useful enough?

##########
File path: cpp/src/arrow/compute/cast.cc
##########
@@ -118,8 +118,86 @@ class CastMetaFunction : public MetaFunction {
 
 }  // namespace
 
+const FunctionDoc struct_doc{"Wrap Arrays into a StructArray",
+                             ("Names of the StructArray's fields are\n"
+                              "specified through StructOptions."),
+                             {},
+                             "StructOptions"};
+
+Result<ValueDescr> StructResolve(KernelContext* ctx,
+                                 const std::vector<ValueDescr>& descrs) {
+  const auto& names = OptionsWrapper<StructOptions>::Get(ctx).field_names;
+  if (names.size() != descrs.size()) {
+    return Status::Invalid("Struct() was passed ", names.size(), " field ", "names but ",
+                           descrs.size(), " arguments");
+  }
+
+  size_t i = 0;
+  FieldVector fields(descrs.size());
+
+  ValueDescr::Shape shape = ValueDescr::SCALAR;
+  for (const ValueDescr& descr : descrs) {
+    if (descr.shape != ValueDescr::SCALAR) {
+      shape = ValueDescr::ARRAY;
+    } else {
+      switch (descr.type->id()) {
+        case Type::EXTENSION:
+        case Type::DENSE_UNION:
+        case Type::SPARSE_UNION:
+          return Status::NotImplemented("Broadcasting scalars of type ", *descr.type);

Review comment:
       It seems this error will be raised even if all inputs are scalars, hence no broadcasting?

##########
File path: cpp/src/arrow/compute/cast.cc
##########
@@ -118,8 +118,86 @@ class CastMetaFunction : public MetaFunction {
 
 }  // namespace
 
+const FunctionDoc struct_doc{"Wrap Arrays into a StructArray",
+                             ("Names of the StructArray's fields are\n"
+                              "specified through StructOptions."),
+                             {},
+                             "StructOptions"};
+
+Result<ValueDescr> StructResolve(KernelContext* ctx,
+                                 const std::vector<ValueDescr>& descrs) {
+  const auto& names = OptionsWrapper<StructOptions>::Get(ctx).field_names;
+  if (names.size() != descrs.size()) {
+    return Status::Invalid("Struct() was passed ", names.size(), " field ", "names but ",
+                           descrs.size(), " arguments");
+  }
+
+  size_t i = 0;
+  FieldVector fields(descrs.size());
+
+  ValueDescr::Shape shape = ValueDescr::SCALAR;
+  for (const ValueDescr& descr : descrs) {
+    if (descr.shape != ValueDescr::SCALAR) {
+      shape = ValueDescr::ARRAY;
+    } else {
+      switch (descr.type->id()) {
+        case Type::EXTENSION:
+        case Type::DENSE_UNION:
+        case Type::SPARSE_UNION:
+          return Status::NotImplemented("Broadcasting scalars of type ", *descr.type);
+        default:
+          break;
+      }
+    }
+
+    fields[i] = field(names[i], descr.type);
+    ++i;
+  }
+
+  return ValueDescr{struct_(std::move(fields)), shape};
+}
+
+void StructExec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  KERNEL_ASSIGN_OR_RAISE(auto descr, ctx, StructResolve(ctx, batch.GetDescriptors()));
+
+  if (descr.shape == ValueDescr::SCALAR) {
+    ScalarVector scalars(batch.num_values());
+    for (int i = 0; i < batch.num_values(); ++i) {
+      scalars[i] = batch[i].scalar();
+    }
+
+    *out =
+        Datum(std::make_shared<StructScalar>(std::move(scalars), std::move(descr.type)));
+    return;
+  }
+
+  ArrayVector arrays(batch.num_values());
+  for (int i = 0; i < batch.num_values(); ++i) {
+    if (batch[i].is_array()) {
+      arrays[i] = batch[i].make_array();
+      continue;
+    }
+
+    KERNEL_ASSIGN_OR_RAISE(
+        arrays[i], ctx,
+        MakeArrayFromScalar(*batch[i].scalar(), batch.length, ctx->memory_pool()));
+  }
+
+  *out = std::make_shared<StructArray>(descr.type, batch.length, std::move(arrays));
+}
+
 void RegisterScalarCast(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::make_shared<CastMetaFunction>()));
+
+  auto struct_function =
+      std::make_shared<ScalarFunction>("struct", Arity::VarArgs(), &struct_doc);

Review comment:
       Should we have a more explicit name? Perhaps "make_struct", "struct_wrap", "struct_of"...

##########
File path: cpp/src/arrow/compute/kernels/scalar_cast_test.cc
##########
@@ -1892,5 +1896,48 @@ TEST_F(TestCast, ExtensionTypeToIntDowncast) {
   ASSERT_OK(UnregisterExtensionType("smallint"));
 }
 
+class TestStruct : public TestBase {
+ public:
+  Result<Datum> Struct(std::vector<Datum> args) {
+    StructOptions opts{field_names};
+    return CallFunction("struct", args, &opts);

Review comment:
       I wouldn't expect to find this test in `scalar_cast_test.cc`. It would be better in its own test file (or a "misc" test file). All scalar kernel tests should be compiled into `arrow-compute-scalar-test` anyway.

##########
File path: cpp/src/arrow/dataset/expression.h
##########
@@ -0,0 +1,234 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This API is EXPERIMENTAL.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/cast.h"
+#include "arrow/dataset/type_fwd.h"
+#include "arrow/dataset/visibility.h"
+#include "arrow/datum.h"
+#include "arrow/result.h"
+#include "arrow/scalar.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/variant.h"
+
+namespace arrow {
+namespace dataset {
+
+/// An unbound expression which maps a single Datum to another Datum.
+/// An expression is one of
+/// - A literal Datum.
+/// - A reference to a single (potentially nested) field of the input Datum.
+/// - A call to a compute function, with arguments specified by other Expressions.
+class ARROW_DS_EXPORT Expression {
+ public:
+  struct Call {
+    std::string function_name;
+    std::vector<Expression> arguments;
+    std::shared_ptr<compute::FunctionOptions> options;
+
+    // post-Bind properties:
+    const compute::Kernel* kernel = NULLPTR;
+    std::shared_ptr<compute::Function> function;
+    std::shared_ptr<compute::KernelState> kernel_state;
+    ValueDescr descr;
+  };
+
+  std::string ToString() const;
+  bool Equals(const Expression& other) const;
+  size_t hash() const;
+  struct Hash {
+    size_t operator()(const Expression& expr) const { return expr.hash(); }
+  };
+
+  /// Bind this expression to the given input type, looking up Kernels and field types.
+  /// Some expression simplification may be performed and implicit casts will be inserted.
+  /// Any state necessary for execution will be initialized and returned.
+  Result<Expression> Bind(ValueDescr in, compute::ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(const Schema& in_schema, compute::ExecContext* = NULLPTR) const;
+
+  // XXX someday
+  // Clone all KernelState in this bound expression. If any function referenced by this
+  // expression has mutable KernelState, it is not safe to execute or apply simplification
+  // passes to it (or copies of it!) from multiple threads. Cloning state produces new
+  // KernelStates where necessary to ensure that Expressions may be manipulated safely
+  // on multiple threads.
+  // Result<ExpressionState> CloneState() const;
+  // Status SetState(ExpressionState);
+
+  /// Return true if all an expression's field references have explicit ValueDescr and all
+  /// of its functions' kernels are looked up.
+  bool IsBound() const;
+
+  /// Return true if this expression is composed only of Scalar literals, field
+  /// references, and calls to ScalarFunctions.
+  bool IsScalarExpression() const;
+
+  /// Return true if this expression is literal and entirely null.
+  bool IsNullLiteral() const;
+
+  /// Return true if this expression could evaluate to true.
+  bool IsSatisfiable() const;
+
+  // XXX someday
+  // Result<PipelineGraph> GetPipelines();
+
+  const Call* call() const;
+  const Datum* literal() const;
+  const FieldRef* field_ref() const;

Review comment:
       Is this for the `Parameter` variant?

##########
File path: cpp/src/arrow/dataset/expression.h
##########
@@ -0,0 +1,234 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This API is EXPERIMENTAL.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "arrow/chunked_array.h"

Review comment:
       Are all those includes required?

##########
File path: cpp/src/arrow/dataset/expression.h
##########
@@ -0,0 +1,234 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This API is EXPERIMENTAL.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/cast.h"
+#include "arrow/dataset/type_fwd.h"
+#include "arrow/dataset/visibility.h"
+#include "arrow/datum.h"
+#include "arrow/result.h"
+#include "arrow/scalar.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/variant.h"
+
+namespace arrow {
+namespace dataset {
+
+/// An unbound expression which maps a single Datum to another Datum.
+/// An expression is one of
+/// - A literal Datum.
+/// - A reference to a single (potentially nested) field of the input Datum.
+/// - A call to a compute function, with arguments specified by other Expressions.
+class ARROW_DS_EXPORT Expression {
+ public:
+  struct Call {
+    std::string function_name;
+    std::vector<Expression> arguments;
+    std::shared_ptr<compute::FunctionOptions> options;
+
+    // post-Bind properties:
+    const compute::Kernel* kernel = NULLPTR;
+    std::shared_ptr<compute::Function> function;
+    std::shared_ptr<compute::KernelState> kernel_state;
+    ValueDescr descr;
+  };
+
+  std::string ToString() const;
+  bool Equals(const Expression& other) const;
+  size_t hash() const;
+  struct Hash {
+    size_t operator()(const Expression& expr) const { return expr.hash(); }
+  };
+
+  /// Bind this expression to the given input type, looking up Kernels and field types.
+  /// Some expression simplification may be performed and implicit casts will be inserted.
+  /// Any state necessary for execution will be initialized and returned.
+  Result<Expression> Bind(ValueDescr in, compute::ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(const Schema& in_schema, compute::ExecContext* = NULLPTR) const;
+
+  // XXX someday
+  // Clone all KernelState in this bound expression. If any function referenced by this
+  // expression has mutable KernelState, it is not safe to execute or apply simplification
+  // passes to it (or copies of it!) from multiple threads. Cloning state produces new
+  // KernelStates where necessary to ensure that Expressions may be manipulated safely
+  // on multiple threads.
+  // Result<ExpressionState> CloneState() const;
+  // Status SetState(ExpressionState);
+
+  /// Return true if all an expression's field references have explicit ValueDescr and all
+  /// of its functions' kernels are looked up.
+  bool IsBound() const;
+
+  /// Return true if this expression is composed only of Scalar literals, field
+  /// references, and calls to ScalarFunctions.
+  bool IsScalarExpression() const;
+
+  /// Return true if this expression is literal and entirely null.
+  bool IsNullLiteral() const;
+
+  /// Return true if this expression could evaluate to true.
+  bool IsSatisfiable() const;
+
+  // XXX someday
+  // Result<PipelineGraph> GetPipelines();
+
+  const Call* call() const;
+  const Datum* literal() const;
+  const FieldRef* field_ref() const;
+
+  ValueDescr descr() const;
+  // XXX someday
+  // NullGeneralization::type nullable() const;
+
+  struct Parameter {
+    FieldRef ref;
+    ValueDescr descr;
+  };
+
+  Expression() = default;
+  explicit Expression(Call call);
+  explicit Expression(Datum literal);
+  explicit Expression(Parameter parameter);
+
+ private:
+  using Impl = util::Variant<Datum, Parameter, Call>;
+  std::shared_ptr<Impl> impl_;
+
+  ARROW_EXPORT friend bool Identical(const Expression& l, const Expression& r);
+
+  ARROW_EXPORT friend void PrintTo(const Expression&, std::ostream*);
+};
+
+inline bool operator==(const Expression& l, const Expression& r) { return l.Equals(r); }
+inline bool operator!=(const Expression& l, const Expression& r) { return !l.Equals(r); }
+
+// Factories
+
+ARROW_DS_EXPORT
+Expression literal(Datum lit);
+
+template <typename Arg>
+Expression literal(Arg&& arg) {
+  return literal(Datum(std::forward<Arg>(arg)));
+}
+
+ARROW_DS_EXPORT
+Expression field_ref(FieldRef ref);
+
+ARROW_DS_EXPORT
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options = NULLPTR);
+
+template <typename Options, typename = typename std::enable_if<std::is_base_of<
+                                compute::FunctionOptions, Options>::value>::type>
+Expression call(std::string function, std::vector<Expression> arguments,
+                Options options) {
+  return call(std::move(function), std::move(arguments),
+              std::make_shared<Options>(std::move(options)));
+}
+
+ARROW_DS_EXPORT
+std::vector<FieldRef> FieldsInExpression(const Expression&);
+
+ARROW_DS_EXPORT
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(

Review comment:
       I'm not sure I understand why this is returned as an unordered_map rather than (far cheaper) vector of pairs.

##########
File path: cpp/src/arrow/dataset/expression.h
##########
@@ -0,0 +1,234 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This API is EXPERIMENTAL.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/cast.h"
+#include "arrow/dataset/type_fwd.h"
+#include "arrow/dataset/visibility.h"
+#include "arrow/datum.h"
+#include "arrow/result.h"
+#include "arrow/scalar.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/variant.h"
+
+namespace arrow {
+namespace dataset {
+
+/// An unbound expression which maps a single Datum to another Datum.
+/// An expression is one of
+/// - A literal Datum.
+/// - A reference to a single (potentially nested) field of the input Datum.
+/// - A call to a compute function, with arguments specified by other Expressions.
+class ARROW_DS_EXPORT Expression {
+ public:
+  struct Call {
+    std::string function_name;
+    std::vector<Expression> arguments;
+    std::shared_ptr<compute::FunctionOptions> options;
+
+    // post-Bind properties:
+    const compute::Kernel* kernel = NULLPTR;
+    std::shared_ptr<compute::Function> function;
+    std::shared_ptr<compute::KernelState> kernel_state;
+    ValueDescr descr;
+  };
+
+  std::string ToString() const;
+  bool Equals(const Expression& other) const;
+  size_t hash() const;
+  struct Hash {
+    size_t operator()(const Expression& expr) const { return expr.hash(); }
+  };
+
+  /// Bind this expression to the given input type, looking up Kernels and field types.
+  /// Some expression simplification may be performed and implicit casts will be inserted.
+  /// Any state necessary for execution will be initialized and returned.
+  Result<Expression> Bind(ValueDescr in, compute::ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(const Schema& in_schema, compute::ExecContext* = NULLPTR) const;
+
+  // XXX someday
+  // Clone all KernelState in this bound expression. If any function referenced by this
+  // expression has mutable KernelState, it is not safe to execute or apply simplification
+  // passes to it (or copies of it!) from multiple threads. Cloning state produces new
+  // KernelStates where necessary to ensure that Expressions may be manipulated safely
+  // on multiple threads.
+  // Result<ExpressionState> CloneState() const;
+  // Status SetState(ExpressionState);
+
+  /// Return true if all an expression's field references have explicit ValueDescr and all
+  /// of its functions' kernels are looked up.
+  bool IsBound() const;
+
+  /// Return true if this expression is composed only of Scalar literals, field
+  /// references, and calls to ScalarFunctions.
+  bool IsScalarExpression() const;
+
+  /// Return true if this expression is literal and entirely null.
+  bool IsNullLiteral() const;
+
+  /// Return true if this expression could evaluate to true.
+  bool IsSatisfiable() const;
+
+  // XXX someday
+  // Result<PipelineGraph> GetPipelines();
+
+  const Call* call() const;
+  const Datum* literal() const;
+  const FieldRef* field_ref() const;
+
+  ValueDescr descr() const;
+  // XXX someday
+  // NullGeneralization::type nullable() const;
+
+  struct Parameter {
+    FieldRef ref;
+    ValueDescr descr;
+  };
+
+  Expression() = default;
+  explicit Expression(Call call);
+  explicit Expression(Datum literal);
+  explicit Expression(Parameter parameter);
+
+ private:
+  using Impl = util::Variant<Datum, Parameter, Call>;
+  std::shared_ptr<Impl> impl_;
+
+  ARROW_EXPORT friend bool Identical(const Expression& l, const Expression& r);
+
+  ARROW_EXPORT friend void PrintTo(const Expression&, std::ostream*);
+};
+
+inline bool operator==(const Expression& l, const Expression& r) { return l.Equals(r); }
+inline bool operator!=(const Expression& l, const Expression& r) { return !l.Equals(r); }
+
+// Factories
+
+ARROW_DS_EXPORT
+Expression literal(Datum lit);
+
+template <typename Arg>
+Expression literal(Arg&& arg) {
+  return literal(Datum(std::forward<Arg>(arg)));
+}
+
+ARROW_DS_EXPORT
+Expression field_ref(FieldRef ref);
+
+ARROW_DS_EXPORT
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options = NULLPTR);
+
+template <typename Options, typename = typename std::enable_if<std::is_base_of<
+                                compute::FunctionOptions, Options>::value>::type>
+Expression call(std::string function, std::vector<Expression> arguments,
+                Options options) {
+  return call(std::move(function), std::move(arguments),
+              std::make_shared<Options>(std::move(options)));
+}
+
+ARROW_DS_EXPORT
+std::vector<FieldRef> FieldsInExpression(const Expression&);
+
+ARROW_DS_EXPORT
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(
+    const Expression& guaranteed_true_predicate);
+
+/// \defgroup expression-passes Functions for modification of Expressions
+///
+/// @{
+///
+/// These operate on bound expressions.
+
+/// Weak canonicalization which establishes guarantees for subsequent passes. Even
+/// equivalent Expressions may result in different canonicalized expressions.
+/// TODO this could be a strong canonicalization
+ARROW_DS_EXPORT
+Result<Expression> Canonicalize(Expression, compute::ExecContext* = NULLPTR);
+
+/// Simplify Expressions based on literal arguments (for example, add(null, x) will always
+/// be null so replace the call with a null literal). Includes early evaluation of all
+/// calls whose arguments are entirely literal.
+ARROW_DS_EXPORT
+Result<Expression> FoldConstants(Expression);
+
+ARROW_DS_EXPORT
+Result<Expression> ReplaceFieldsWithKnownValues(
+    const std::unordered_map<FieldRef, Datum, FieldRef::Hash>& known_values, Expression);

Review comment:
       Same question wrt. unordered_map. The implementation may use unordered_map internally but I'm not sure it makes sense to expose this detail in the API.

##########
File path: cpp/src/arrow/dataset/partition.cc
##########
@@ -573,5 +530,192 @@ Result<std::shared_ptr<Schema>> PartitioningOrFactory::GetOrInferSchema(
   return factory()->Inspect(paths);
 }
 
+// Transform an array of counts to offsets which will divide a ListArray
+// into an equal number of slices with corresponding lengths.
+inline Result<std::shared_ptr<Array>> CountsToOffsets(
+    std::shared_ptr<Int64Array> counts) {
+  Int32Builder offset_builder;
+  RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1));
+  offset_builder.UnsafeAppend(0);
+
+  for (int64_t i = 0; i < counts->length(); ++i) {
+    DCHECK_NE(counts->Value(i), 0);
+    auto next_offset = static_cast<int32_t>(offset_builder[i] + counts->Value(i));
+    offset_builder.UnsafeAppend(next_offset);
+  }
+
+  std::shared_ptr<Array> offsets;
+  RETURN_NOT_OK(offset_builder.Finish(&offsets));
+  return offsets;
+}
+
+// Helper for simultaneous dictionary encoding of multiple arrays.
+//
+// The fused dictionary is the Cartesian product of the individual dictionaries.
+// For example given two arrays A, B where A has unique values ["ex", "why"]
+// and B has unique values [0, 1] the fused dictionary is the set of tuples
+// [["ex", 0], ["ex", 1], ["why", 0], ["ex", 1]].
+//
+// TODO(bkietz) this capability belongs in an Action of the hash kernels, where
+// it can be used to group aggregates without materializing a grouped batch.
+// For the purposes of writing we need the materialized grouped batch anyway
+// since no Writers accept a selection vector.
+class StructDictionary {
+ public:
+  struct Encoded {
+    std::shared_ptr<Int32Array> indices;
+    std::shared_ptr<StructDictionary> dictionary;
+  };
+
+  static Result<Encoded> Encode(const ArrayVector& columns) {
+    Encoded out{nullptr, std::make_shared<StructDictionary>()};
+
+    for (const auto& column : columns) {
+      if (column->null_count() != 0) {
+        return Status::NotImplemented("Grouping on a field with nulls");
+      }
+
+      RETURN_NOT_OK(out.dictionary->AddOne(column, &out.indices));
+    }
+
+    return out;
+  }
+
+  Result<std::shared_ptr<StructArray>> Decode(std::shared_ptr<Int32Array> fused_indices,
+                                              FieldVector fields) {
+    std::vector<Int32Builder> builders(dictionaries_.size());
+    for (Int32Builder& b : builders) {
+      RETURN_NOT_OK(b.Resize(fused_indices->length()));
+    }
+
+    std::vector<int32_t> codes(dictionaries_.size());
+    for (int64_t i = 0; i < fused_indices->length(); ++i) {
+      Expand(fused_indices->Value(i), codes.data());
+
+      auto builder_it = builders.begin();
+      for (int32_t index : codes) {
+        builder_it++->UnsafeAppend(index);
+      }
+    }
+
+    ArrayVector columns(dictionaries_.size());
+    for (size_t i = 0; i < dictionaries_.size(); ++i) {
+      std::shared_ptr<ArrayData> indices;
+      RETURN_NOT_OK(builders[i].FinishInternal(&indices));
+
+      ARROW_ASSIGN_OR_RAISE(Datum column, compute::Take(dictionaries_[i], indices));
+      columns[i] = column.make_array();
+    }
+
+    return StructArray::Make(std::move(columns), std::move(fields));
+  }
+
+ private:
+  Status AddOne(Datum column, std::shared_ptr<Int32Array>* fused_indices) {
+    ArrayData* encoded;
+    if (column.type()->id() != Type::DICTIONARY) {
+      ARROW_ASSIGN_OR_RAISE(column, compute::DictionaryEncode(column));
+    }
+    encoded = column.mutable_array();
+
+    auto indices =
+        std::make_shared<Int32Array>(encoded->length, std::move(encoded->buffers[1]));
+
+    dictionaries_.push_back(MakeArray(std::move(encoded->dictionary)));
+    auto dictionary_size = static_cast<int32_t>(dictionaries_.back()->length());
+
+    if (*fused_indices == nullptr) {
+      *fused_indices = std::move(indices);
+      size_ = dictionary_size;
+      return Status::OK();
+    }
+
+    // It's useful to think about the case where each of dictionaries_ has size 10.
+    // In this case the decimal digit in the ones place is the code in dictionaries_[0],
+    // the tens place corresponds to dictionaries_[1], etc.
+    // The incumbent indices must be shifted to the hundreds place so as not to collide.
+    ARROW_ASSIGN_OR_RAISE(Datum new_fused_indices,
+                          compute::Multiply(indices, MakeScalar(size_)));
+
+    ARROW_ASSIGN_OR_RAISE(new_fused_indices,
+                          compute::Add(new_fused_indices, *fused_indices));
+
+    *fused_indices = checked_pointer_cast<Int32Array>(new_fused_indices.make_array());
+
+    // XXX should probably cap this at 2**15 or so
+    ARROW_CHECK(!internal::MultiplyWithOverflow(size_, dictionary_size, &size_));

Review comment:
       Why not return an error instead of aborting?
   (also, why the comment "cap at 2**15"?).

##########
File path: cpp/src/arrow/type.cc
##########
@@ -1039,13 +1042,31 @@ Result<std::shared_ptr<Field>> FieldPath::Get(const FieldVector& fields) const {
 
 Result<std::shared_ptr<Array>> FieldPath::Get(const RecordBatch& batch) const {
   ARROW_ASSIGN_OR_RAISE(auto data, FieldPathGetImpl::Get(this, batch.column_data()));
-  return MakeArray(data);
+  return MakeArray(std::move(data));
 }
 
 Result<std::shared_ptr<ChunkedArray>> FieldPath::Get(const Table& table) const {
   return FieldPathGetImpl::Get(this, table.columns());
 }
 
+Result<std::shared_ptr<Array>> FieldPath::Get(const Array& array) const {
+  ARROW_ASSIGN_OR_RAISE(auto data, Get(*array.data()));
+  return MakeArray(std::move(data));
+}
+
+Result<std::shared_ptr<ArrayData>> FieldPath::Get(const ArrayData& data) const {
+  return FieldPathGetImpl::Get(this, data.child_data);
+}
+
+Result<std::shared_ptr<ChunkedArray>> FieldPath::Get(const ChunkedArray& array) const {
+  FieldPath prefixed_with_0 = *this;
+  prefixed_with_0.indices_.insert(prefixed_with_0.indices_.begin(), 0);
+
+  ChunkedArrayVector vec;
+  vec.emplace_back(const_cast<ChunkedArray*>(&array), [](...) {});

Review comment:
       I can't parse what this does. Would you mind leaving a comment or perhaps finding a nicer spelling?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {

Review comment:
       Shouldn't this go into `type.cc`?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{
+        {"equal", EQUAL},     {"not_equal", NOT_EQUAL},
+        {"less", LESS},       {"less_equal", LESS_EQUAL},
+        {"greater", GREATER}, {"greater_equal", GREATER_EQUAL},
+    };
+
+    auto it = flipped_comparisons.find(function);
+    return it != flipped_comparisons.end() ? &it->second : nullptr;
+  }
+
+  static const type* Get(const Expression& expr) {
+    if (auto call = expr.call()) {
+      return Comparison::Get(call->function_name);
+    }
+    return nullptr;
+  }
+
+  // Execute a simple Comparison between scalars, casting the RHS if types disagree
+  static Result<type> Execute(Datum l, Datum r) {
+    if (!l.is_scalar() || !r.is_scalar()) {
+      return Status::Invalid("Cannot Execute Comparison on non-scalars");
+    }
+
+    if (!l.type()->Equals(r.type())) {
+      ARROW_ASSIGN_OR_RAISE(r, compute::Cast(r, l.type()));
+    }
+
+    std::vector<Datum> arguments{std::move(l), std::move(r)};
+
+    ARROW_ASSIGN_OR_RAISE(auto equal, compute::CallFunction("equal", arguments));
+
+    if (!equal.scalar()->is_valid) return NA;
+    if (equal.scalar_as<BooleanScalar>().value) return EQUAL;
+
+    ARROW_ASSIGN_OR_RAISE(auto less, compute::CallFunction("less", arguments));
+
+    if (!less.scalar()->is_valid) return NA;
+    return less.scalar_as<BooleanScalar>().value ? LESS : GREATER;
+  }
+
+  static type GetFlipped(type op) {
+    switch (op) {
+      case NA:
+        return NA;
+      case EQUAL:
+        return EQUAL;
+      case LESS:
+        return GREATER;
+      case GREATER:
+        return LESS;
+      case NOT_EQUAL:
+        return NOT_EQUAL;
+      case LESS_EQUAL:
+        return GREATER_EQUAL;
+      case GREATER_EQUAL:
+        return LESS_EQUAL;
+    }
+    DCHECK(false);
+    return NA;
+  }
+
+  static std::string GetName(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "equal";
+      case LESS:
+        return "less";
+      case GREATER:
+        return "greater";
+      case NOT_EQUAL:
+        return "not_equal";
+      case LESS_EQUAL:
+        return "less_equal";
+      case GREATER_EQUAL:
+        return "greater_equal";
+    }
+    DCHECK(false);
+    return "na";
+  }
+
+  static std::string GetOp(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "==";
+      case LESS:
+        return "<";
+      case GREATER:
+        return ">";
+      case NOT_EQUAL:
+        return "!=";
+      case LESS_EQUAL:
+        return "<=";
+      case GREATER_EQUAL:
+        return ">=";
+    }
+    DCHECK(false);
+    return "";
+  }
+};
+
+inline const compute::CastOptions* GetCastOptions(const Expression::Call& call) {
+  if (call.function_name != "cast") return nullptr;
+  return checked_cast<const compute::CastOptions*>(call.options.get());
+}
+
+inline bool IsSetLookup(const std::string& function) {
+  return function == "is_in" || function == "index_in";
+}
+
+inline bool IsSameTypesBinary(const std::string& function) {
+  if (Comparison::Get(function)) return true;
+
+  static std::unordered_set<std::string> set{"add", "subtract", "multiply", "divide"};
+
+  return set.find(function) != set.end();
+}
+
+inline const compute::SetLookupOptions* GetSetLookupOptions(
+    const Expression::Call& call) {
+  if (!IsSetLookup(call.function_name)) return nullptr;
+  return checked_cast<const compute::SetLookupOptions*>(call.options.get());
+}
+
+inline const compute::StructOptions* GetStructOptions(const Expression::Call& call) {
+  if (call.function_name != "struct") return nullptr;
+  return checked_cast<const compute::StructOptions*>(call.options.get());
+}
+
+inline const compute::StrptimeOptions* GetStrptimeOptions(const Expression::Call& call) {
+  if (call.function_name != "strptime") return nullptr;
+  return checked_cast<const compute::StrptimeOptions*>(call.options.get());
+}
+
+inline const std::shared_ptr<DataType>& GetDictionaryValueType(

Review comment:
       It seems there's no point in returning a const-ref as all callers take a copy.

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;

Review comment:
       Might check this first before recursing into the expression tree?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{
+        {"equal", EQUAL},     {"not_equal", NOT_EQUAL},
+        {"less", LESS},       {"less_equal", LESS_EQUAL},
+        {"greater", GREATER}, {"greater_equal", GREATER_EQUAL},
+    };
+
+    auto it = flipped_comparisons.find(function);
+    return it != flipped_comparisons.end() ? &it->second : nullptr;
+  }
+
+  static const type* Get(const Expression& expr) {
+    if (auto call = expr.call()) {
+      return Comparison::Get(call->function_name);
+    }
+    return nullptr;
+  }
+
+  // Execute a simple Comparison between scalars, casting the RHS if types disagree
+  static Result<type> Execute(Datum l, Datum r) {
+    if (!l.is_scalar() || !r.is_scalar()) {
+      return Status::Invalid("Cannot Execute Comparison on non-scalars");
+    }
+
+    if (!l.type()->Equals(r.type())) {
+      ARROW_ASSIGN_OR_RAISE(r, compute::Cast(r, l.type()));
+    }
+
+    std::vector<Datum> arguments{std::move(l), std::move(r)};
+
+    ARROW_ASSIGN_OR_RAISE(auto equal, compute::CallFunction("equal", arguments));
+
+    if (!equal.scalar()->is_valid) return NA;
+    if (equal.scalar_as<BooleanScalar>().value) return EQUAL;
+
+    ARROW_ASSIGN_OR_RAISE(auto less, compute::CallFunction("less", arguments));
+
+    if (!less.scalar()->is_valid) return NA;
+    return less.scalar_as<BooleanScalar>().value ? LESS : GREATER;
+  }
+
+  static type GetFlipped(type op) {
+    switch (op) {
+      case NA:
+        return NA;
+      case EQUAL:
+        return EQUAL;
+      case LESS:
+        return GREATER;
+      case GREATER:
+        return LESS;
+      case NOT_EQUAL:
+        return NOT_EQUAL;
+      case LESS_EQUAL:
+        return GREATER_EQUAL;
+      case GREATER_EQUAL:
+        return LESS_EQUAL;
+    }
+    DCHECK(false);
+    return NA;
+  }
+
+  static std::string GetName(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "equal";
+      case LESS:
+        return "less";
+      case GREATER:
+        return "greater";
+      case NOT_EQUAL:
+        return "not_equal";
+      case LESS_EQUAL:
+        return "less_equal";
+      case GREATER_EQUAL:
+        return "greater_equal";
+    }
+    DCHECK(false);
+    return "na";
+  }
+
+  static std::string GetOp(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "==";
+      case LESS:
+        return "<";
+      case GREATER:
+        return ">";
+      case NOT_EQUAL:
+        return "!=";
+      case LESS_EQUAL:
+        return "<=";
+      case GREATER_EQUAL:
+        return ">=";
+    }
+    DCHECK(false);
+    return "";
+  }
+};
+
+inline const compute::CastOptions* GetCastOptions(const Expression::Call& call) {
+  if (call.function_name != "cast") return nullptr;
+  return checked_cast<const compute::CastOptions*>(call.options.get());
+}
+
+inline bool IsSetLookup(const std::string& function) {
+  return function == "is_in" || function == "index_in";
+}
+
+inline bool IsSameTypesBinary(const std::string& function) {
+  if (Comparison::Get(function)) return true;
+
+  static std::unordered_set<std::string> set{"add", "subtract", "multiply", "divide"};
+
+  return set.find(function) != set.end();
+}
+
+inline const compute::SetLookupOptions* GetSetLookupOptions(
+    const Expression::Call& call) {
+  if (!IsSetLookup(call.function_name)) return nullptr;
+  return checked_cast<const compute::SetLookupOptions*>(call.options.get());
+}
+
+inline const compute::StructOptions* GetStructOptions(const Expression::Call& call) {
+  if (call.function_name != "struct") return nullptr;
+  return checked_cast<const compute::StructOptions*>(call.options.get());
+}
+
+inline const compute::StrptimeOptions* GetStrptimeOptions(const Expression::Call& call) {
+  if (call.function_name != "strptime") return nullptr;
+  return checked_cast<const compute::StrptimeOptions*>(call.options.get());
+}
+
+inline const std::shared_ptr<DataType>& GetDictionaryValueType(
+    const std::shared_ptr<DataType>& type) {
+  if (type && type->id() == Type::DICTIONARY) {
+    return checked_cast<const DictionaryType&>(*type).value_type();
+  }
+  static std::shared_ptr<DataType> null;
+  return null;
+}
+
+inline Status EnsureNotDictionary(ValueDescr* descr) {
+  if (auto value_type = GetDictionaryValueType(descr->type)) {
+    descr->type = std::move(value_type);
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Datum* datum) {
+  if (datum->type()->id() == Type::DICTIONARY) {
+    const auto& type = checked_cast<const DictionaryType&>(*datum->type()).value_type();
+    ARROW_ASSIGN_OR_RAISE(*datum, compute::Cast(*datum, type));
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Expression::Call* call) {
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto new_options = *options;
+    RETURN_NOT_OK(EnsureNotDictionary(&new_options.value_set));
+    call->options.reset(new compute::SetLookupOptions(std::move(new_options)));
+  }
+  return Status::OK();
+}
+
+inline Result<std::shared_ptr<StructScalar>> FunctionOptionsToStructScalar(
+    const Expression::Call& call) {
+  if (call.options == nullptr) {
+    return nullptr;
+  }
+
+  auto Finish = [](ScalarVector values, std::vector<std::string> names) {
+    FieldVector fields(names.size());
+    for (size_t i = 0; i < fields.size(); ++i) {
+      fields[i] = field(std::move(names[i]), values[i]->type);
+    }
+    return std::make_shared<StructScalar>(std::move(values), struct_(std::move(fields)));
+  };
+
+  if (auto options = GetSetLookupOptions(call)) {
+    if (!options->value_set.is_array()) {
+      return Status::NotImplemented("chunked value_set");
+    }
+    return Finish(
+        {
+            std::make_shared<ListScalar>(options->value_set.make_array()),
+            MakeScalar(options->skip_nulls),
+        },
+        {"value_set", "skip_nulls"});
+  }
+
+  if (call.function_name == "cast") {
+    auto options = checked_cast<const compute::CastOptions*>(call.options.get());
+    return Finish(
+        {
+            MakeNullScalar(options->to_type),
+            MakeScalar(options->allow_int_overflow),
+            MakeScalar(options->allow_time_truncate),
+            MakeScalar(options->allow_time_overflow),
+            MakeScalar(options->allow_decimal_truncate),
+            MakeScalar(options->allow_float_truncate),
+            MakeScalar(options->allow_invalid_utf8),
+        },
+        {
+            "to_type_holder",
+            "allow_int_overflow",
+            "allow_time_truncate",
+            "allow_time_overflow",
+            "allow_decimal_truncate",
+            "allow_float_truncate",
+            "allow_invalid_utf8",
+        });
+  }
+
+  return Status::NotImplemented("conversion of options for ", call.function_name);
+}
+
+inline Status FunctionOptionsFromStructScalar(const StructScalar* repr,
+                                              Expression::Call* call) {
+  if (repr == nullptr) {
+    call->options = nullptr;
+    return Status::OK();
+  }
+
+  if (IsSetLookup(call->function_name)) {
+    ARROW_ASSIGN_OR_RAISE(auto value_set, repr->field("value_set"));
+    ARROW_ASSIGN_OR_RAISE(auto skip_nulls, repr->field("skip_nulls"));
+    call->options = std::make_shared<compute::SetLookupOptions>(
+        checked_cast<const ListScalar&>(*value_set).value,
+        checked_cast<const BooleanScalar&>(*skip_nulls).value);
+    return Status::OK();
+  }
+
+  if (call->function_name == "cast") {
+    auto options = std::make_shared<compute::CastOptions>();
+    ARROW_ASSIGN_OR_RAISE(auto to_type_holder, repr->field("to_type_holder"));
+    options->to_type = to_type_holder->type;
+
+    int i = 1;
+    for (bool* opt : {
+             &options->allow_int_overflow,
+             &options->allow_time_truncate,
+             &options->allow_time_overflow,
+             &options->allow_decimal_truncate,
+             &options->allow_float_truncate,
+             &options->allow_invalid_utf8,
+         }) {
+      *opt = checked_cast<const BooleanScalar&>(*repr->value[i++]).value;
+    }
+
+    call->options = std::move(options);
+    return Status::OK();
+  }
+
+  return Status::NotImplemented("conversion of options for ", call->function_name);
+}
+
+struct FlattenedAssociativeChain {
+  bool was_left_folded = true;
+  std::vector<Expression> exprs, fringe;
+
+  explicit FlattenedAssociativeChain(Expression expr) : exprs{std::move(expr)} {
+    auto call = CallNotNull(exprs.back());
+    fringe = call->arguments;
+
+    auto it = fringe.begin();
+
+    while (it != fringe.end()) {
+      auto sub_call = it->call();
+      if (!sub_call || sub_call->function_name != call->function_name) {
+        ++it;
+        continue;
+      }
+
+      if (it != fringe.begin()) {
+        was_left_folded = false;
+      }
+
+      exprs.push_back(std::move(*it));
+      it = fringe.erase(it);
+      it = fringe.insert(it, sub_call->arguments.begin(), sub_call->arguments.end());
+      // NB: no increment so we hit sub_call's first argument next iteration
+    }
+
+    DCHECK(std::all_of(exprs.begin(), exprs.end(), [](const Expression& expr) {
+      return CallNotNull(expr)->options == nullptr;
+    }));
+  }
+};
+
+inline Result<std::shared_ptr<compute::Function>> GetFunction(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (call.function_name != "cast") {
+    return exec_context->func_registry()->GetFunction(call.function_name);
+  }
+  // XXX this special case is strange; why not make "cast" a ScalarFunction?
+  const auto& to_type = checked_cast<const compute::CastOptions&>(*call.options).to_type;
+  return compute::GetCastFunction(to_type);
+}
+
+template <typename PreVisit, typename PostVisitCall>
+Result<Expression> Modify(Expression expr, const PreVisit& pre,

Review comment:
       Add a comment/docstring?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},

Review comment:
       Why are you not passing `expr` directly here?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {

Review comment:
       Why is this useful?
   Also note that "immutable state" is kind of self-contradictory. We probably need to stuff precomputed data elsewhere than in the kernel state... See ARROW-10556, but the solution may be something else than a caching layer.

##########
File path: cpp/src/arrow/compute/cast.cc
##########
@@ -118,8 +118,86 @@ class CastMetaFunction : public MetaFunction {
 
 }  // namespace
 
+const FunctionDoc struct_doc{"Wrap Arrays into a StructArray",
+                             ("Names of the StructArray's fields are\n"
+                              "specified through StructOptions."),
+                             {},
+                             "StructOptions"};
+
+Result<ValueDescr> StructResolve(KernelContext* ctx,
+                                 const std::vector<ValueDescr>& descrs) {
+  const auto& names = OptionsWrapper<StructOptions>::Get(ctx).field_names;
+  if (names.size() != descrs.size()) {
+    return Status::Invalid("Struct() was passed ", names.size(), " field ", "names but ",
+                           descrs.size(), " arguments");
+  }
+
+  size_t i = 0;
+  FieldVector fields(descrs.size());
+
+  ValueDescr::Shape shape = ValueDescr::SCALAR;
+  for (const ValueDescr& descr : descrs) {
+    if (descr.shape != ValueDescr::SCALAR) {
+      shape = ValueDescr::ARRAY;
+    } else {
+      switch (descr.type->id()) {
+        case Type::EXTENSION:

Review comment:
       Hmm... shouldn't broadcasting an extension scalar be as simple as broadcasting the underlying storage scalar?

##########
File path: cpp/src/arrow/compute/kernels/scalar_cast_internal.cc
##########
@@ -160,16 +172,15 @@ void UnpackDictionary(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
     return;
   }
 
-  Result<Datum> result = Take(Datum(dict_arr.dictionary()), Datum(dict_arr.indices()),
-                              /*options=*/TakeOptions::Defaults(), ctx->exec_context());
-  if (!result.ok()) {
-    ctx->SetStatus(result.status());
-    return;
-  }
-  *out = *result;
+  return Finish(Take(Datum(dict_arr.dictionary()), Datum(dict_arr.indices()),

Review comment:
       `KERNEL_ASSIGN_OR_RAISE` as well?

##########
File path: cpp/src/arrow/compute/cast.cc
##########
@@ -135,7 +213,7 @@ CastFunction::CastFunction(std::string name, Type::type out_type)
   impl_->out_type = out_type;
 }
 
-CastFunction::~CastFunction() {}
+CastFunction::~CastFunction() = default;

Review comment:
       Is there a particular reason for preferring one over the other, btw?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();
+
+  compute::KernelContext kernel_context(exec_context);
+  kernel_context.SetState(call->kernel_state.get());
+
+  auto kernel = call->kernel;
+  auto descrs = GetDescriptors(arguments);
+  auto options = call->options.get();
+  RETURN_NOT_OK(executor->Init(&kernel_context, {kernel, descrs, options}));
+
+  auto listener = std::make_shared<compute::detail::DatumAccumulator>();
+  RETURN_NOT_OK(executor->Execute(arguments, listener.get()));
+  return executor->WrapResults(arguments, listener->values());
+}
+
+std::array<std::pair<const Expression&, const Expression&>, 2>
+ArgumentsAndFlippedArguments(const Expression::Call& call) {
+  DCHECK_EQ(call.arguments.size(), 2);
+  return {std::pair<const Expression&, const Expression&>{call.arguments[0],
+                                                          call.arguments[1]},
+          std::pair<const Expression&, const Expression&>{call.arguments[1],
+                                                          call.arguments[0]}};
+}
+
+template <typename BinOp, typename It,
+          typename Out = typename std::iterator_traits<It>::value_type>
+util::optional<Out> FoldLeft(It begin, It end, const BinOp& bin_op) {
+  if (begin == end) return util::nullopt;
+
+  Out folded = std::move(*begin++);
+  while (begin != end) {
+    folded = bin_op(std::move(folded), std::move(*begin++));
+  }
+  return folded;
+}
+
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  if (call.function && call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
+  }
+  return util::nullopt;
+}
+
+bool DefinitelyNotNull(const Expression& expr) {
+  DCHECK(expr.IsBound());
+
+  if (expr.literal()) {
+    return !expr.IsNullLiteral();
+  }
+
+  if (expr.field_ref()) return false;
+
+  auto call = CallNotNull(expr);
+  if (auto null_handling = GetNullHandling(*call)) {
+    if (null_handling == compute::NullHandling::OUTPUT_NOT_NULL) {
+      return true;
+    }
+    if (null_handling == compute::NullHandling::INTERSECTION) {
+      return std::all_of(call->arguments.begin(), call->arguments.end(),
+                         DefinitelyNotNull);
+    }
+  }
+
+  return false;
+}
+
+std::vector<FieldRef> FieldsInExpression(const Expression& expr) {
+  if (auto lit = expr.literal()) return {};
+
+  if (auto ref = expr.field_ref()) {
+    return {*ref};
+  }
+
+  std::vector<FieldRef> fields;
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    auto argument_fields = FieldsInExpression(arg);
+    std::move(argument_fields.begin(), argument_fields.end(), std::back_inserter(fields));
+  }
+  return fields;
+}
+
+Result<Expression> FoldConstants(Expression expr) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [](Expression expr, ...) -> Result<Expression> {
+        auto call = CallNotNull(expr);
+        if (std::all_of(call->arguments.begin(), call->arguments.end(),
+                        [](const Expression& argument) { return argument.literal(); })) {
+          // all arguments are literal; we can evaluate this subexpression *now*
+          static const Datum ignored_input;
+          ARROW_ASSIGN_OR_RAISE(Datum constant,
+                                ExecuteScalarExpression(expr, ignored_input));
+
+          return literal(std::move(constant));
+        }
+
+        // XXX the following should probably be in a registry of passes instead
+        // of inline
+
+        if (GetNullHandling(*call) == compute::NullHandling::INTERSECTION) {
+          // kernels which always produce intersected validity can be resolved
+          // to null *now* if any of their inputs is a null literal
+          for (const auto& argument : call->arguments) {
+            if (argument.IsNullLiteral()) {
+              return argument;
+            }
+          }
+        }
+
+        if (call->function_name == "and_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // true and x == x
+            if (args.first == literal(true)) return args.second;
+
+            // false and x == false
+            if (args.first == literal(false)) return args.first;
+
+            // x and x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        if (call->function_name == "or_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // false or x == x
+            if (args.first == literal(false)) return args.second;
+
+            // true or x == true
+            if (args.first == literal(true)) return args.first;
+
+            // x or x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        return expr;
+      });
+}
+
+inline std::vector<Expression> GuaranteeConjunctionMembers(
+    const Expression& guaranteed_true_predicate) {
+  auto guarantee = guaranteed_true_predicate.call();
+  if (!guarantee || guarantee->function_name != "and_kleene") {
+    return {guaranteed_true_predicate};
+  }
+  return FlattenedAssociativeChain(guaranteed_true_predicate).fringe;
+}
+
+// Conjunction members which are represented in known_values are erased from
+// conjunction_members
+Status ExtractKnownFieldValuesImpl(
+    std::vector<Expression>* conjunction_members,
+    std::unordered_map<FieldRef, Datum, FieldRef::Hash>* known_values) {
+  auto unconsumed_end =
+      std::partition(conjunction_members->begin(), conjunction_members->end(),
+                     [](const Expression& expr) {
+                       // search for an equality conditions between a field and a literal
+                       auto call = expr.call();
+                       if (!call) return true;
+
+                       if (call->function_name == "equal") {
+                         auto ref = call->arguments[0].field_ref();
+                         auto lit = call->arguments[1].literal();
+                         return !(ref && lit);

Review comment:
       I suppose the caller should be careful about argument order :-)

##########
File path: cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc
##########
@@ -256,14 +251,45 @@ struct CastFunctor<Date64Type, Date32Type> {
 template <>
 struct CastFunctor<Date32Type, Date64Type> {
   static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
-    // TODO: Make this work on scalar inputs
     DCHECK_EQ(batch[0].kind(), Datum::ARRAY);
 
     ShiftTime<int64_t, int32_t>(ctx, util::DIVIDE, kMillisecondsInDay, *batch[0].array(),
                                 out->mutable_array());
   }
 };
 
+// ----------------------------------------------------------------------
+// date32, date64 to timestamp
+
+template <>
+struct CastFunctor<TimestampType, Date32Type> {
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    DCHECK_EQ(batch[0].kind(), Datum::ARRAY);
+
+    const auto& out_type = checked_cast<const TimestampType&>(*out->type());
+    // get conversion SECOND -> unit
+    auto conversion = util::GetTimestampConversion(TimeUnit::SECOND, out_type.unit());
+    DCHECK_EQ(conversion.first, util::MULTIPLY);
+
+    // multiply to achieve days -> unit
+    conversion.second *= kMillisecondsInDay / 1000;
+    ShiftTime<int32_t, int64_t>(ctx, util::MULTIPLY, conversion.second, *batch[0].array(),
+                                out->mutable_array());
+  }
+};
+
+template <>
+struct CastFunctor<TimestampType, Date64Type> {
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    DCHECK_EQ(batch[0].kind(), Datum::ARRAY);
+
+    const auto& out_type = checked_cast<const TimestampType&>(*out->type());
+    auto conversion = util::GetTimestampConversion(TimeUnit::MILLI, out_type.unit());
+    ShiftTime<int64_t, int64_t>(ctx, util::MULTIPLY, conversion.second, *batch[0].array(),

Review comment:
       Add a check on `conversion.first` as above?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();
+
+  compute::KernelContext kernel_context(exec_context);
+  kernel_context.SetState(call->kernel_state.get());
+
+  auto kernel = call->kernel;
+  auto descrs = GetDescriptors(arguments);
+  auto options = call->options.get();
+  RETURN_NOT_OK(executor->Init(&kernel_context, {kernel, descrs, options}));
+
+  auto listener = std::make_shared<compute::detail::DatumAccumulator>();
+  RETURN_NOT_OK(executor->Execute(arguments, listener.get()));
+  return executor->WrapResults(arguments, listener->values());
+}
+
+std::array<std::pair<const Expression&, const Expression&>, 2>
+ArgumentsAndFlippedArguments(const Expression::Call& call) {
+  DCHECK_EQ(call.arguments.size(), 2);
+  return {std::pair<const Expression&, const Expression&>{call.arguments[0],
+                                                          call.arguments[1]},
+          std::pair<const Expression&, const Expression&>{call.arguments[1],
+                                                          call.arguments[0]}};
+}
+
+template <typename BinOp, typename It,
+          typename Out = typename std::iterator_traits<It>::value_type>
+util::optional<Out> FoldLeft(It begin, It end, const BinOp& bin_op) {
+  if (begin == end) return util::nullopt;
+
+  Out folded = std::move(*begin++);
+  while (begin != end) {
+    folded = bin_op(std::move(folded), std::move(*begin++));
+  }
+  return folded;
+}
+
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  if (call.function && call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
+  }
+  return util::nullopt;
+}
+
+bool DefinitelyNotNull(const Expression& expr) {
+  DCHECK(expr.IsBound());
+
+  if (expr.literal()) {
+    return !expr.IsNullLiteral();
+  }
+
+  if (expr.field_ref()) return false;
+
+  auto call = CallNotNull(expr);
+  if (auto null_handling = GetNullHandling(*call)) {
+    if (null_handling == compute::NullHandling::OUTPUT_NOT_NULL) {
+      return true;
+    }
+    if (null_handling == compute::NullHandling::INTERSECTION) {
+      return std::all_of(call->arguments.begin(), call->arguments.end(),
+                         DefinitelyNotNull);
+    }
+  }
+
+  return false;
+}
+
+std::vector<FieldRef> FieldsInExpression(const Expression& expr) {
+  if (auto lit = expr.literal()) return {};
+
+  if (auto ref = expr.field_ref()) {
+    return {*ref};
+  }
+
+  std::vector<FieldRef> fields;
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    auto argument_fields = FieldsInExpression(arg);
+    std::move(argument_fields.begin(), argument_fields.end(), std::back_inserter(fields));
+  }
+  return fields;
+}
+
+Result<Expression> FoldConstants(Expression expr) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [](Expression expr, ...) -> Result<Expression> {
+        auto call = CallNotNull(expr);
+        if (std::all_of(call->arguments.begin(), call->arguments.end(),
+                        [](const Expression& argument) { return argument.literal(); })) {
+          // all arguments are literal; we can evaluate this subexpression *now*
+          static const Datum ignored_input;
+          ARROW_ASSIGN_OR_RAISE(Datum constant,
+                                ExecuteScalarExpression(expr, ignored_input));
+
+          return literal(std::move(constant));
+        }
+
+        // XXX the following should probably be in a registry of passes instead
+        // of inline
+
+        if (GetNullHandling(*call) == compute::NullHandling::INTERSECTION) {
+          // kernels which always produce intersected validity can be resolved
+          // to null *now* if any of their inputs is a null literal
+          for (const auto& argument : call->arguments) {
+            if (argument.IsNullLiteral()) {
+              return argument;
+            }
+          }
+        }
+
+        if (call->function_name == "and_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // true and x == x
+            if (args.first == literal(true)) return args.second;
+
+            // false and x == false
+            if (args.first == literal(false)) return args.first;
+
+            // x and x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        if (call->function_name == "or_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // false or x == x
+            if (args.first == literal(false)) return args.second;
+
+            // true or x == true
+            if (args.first == literal(true)) return args.first;
+
+            // x or x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        return expr;
+      });
+}
+
+inline std::vector<Expression> GuaranteeConjunctionMembers(
+    const Expression& guaranteed_true_predicate) {
+  auto guarantee = guaranteed_true_predicate.call();
+  if (!guarantee || guarantee->function_name != "and_kleene") {
+    return {guaranteed_true_predicate};
+  }
+  return FlattenedAssociativeChain(guaranteed_true_predicate).fringe;
+}
+
+// Conjunction members which are represented in known_values are erased from
+// conjunction_members
+Status ExtractKnownFieldValuesImpl(
+    std::vector<Expression>* conjunction_members,
+    std::unordered_map<FieldRef, Datum, FieldRef::Hash>* known_values) {
+  auto unconsumed_end =
+      std::partition(conjunction_members->begin(), conjunction_members->end(),
+                     [](const Expression& expr) {
+                       // search for an equality conditions between a field and a literal
+                       auto call = expr.call();
+                       if (!call) return true;
+
+                       if (call->function_name == "equal") {
+                         auto ref = call->arguments[0].field_ref();
+                         auto lit = call->arguments[1].literal();
+                         return !(ref && lit);
+                       }
+
+                       return true;
+                     });
+
+  for (auto it = unconsumed_end; it != conjunction_members->end(); ++it) {
+    auto call = CallNotNull(*it);
+
+    auto ref = call->arguments[0].field_ref();
+    auto lit = call->arguments[1].literal();
+
+    auto it_success = known_values->emplace(*ref, *lit);
+    if (it_success.second) continue;
+
+    // A value was already known for ref; check it
+    auto ref_lit = it_success.first;
+    if (*lit != ref_lit->second) {
+      return Status::Invalid("Conflicting guarantees: (", ref->ToString(),
+                             " == ", lit->ToString(), ") vs (", ref->ToString(),
+                             " == ", ref_lit->second.ToString());
+    }
+  }
+
+  conjunction_members->erase(unconsumed_end, conjunction_members->end());
+
+  return Status::OK();
+}
+
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(
+    const Expression& guaranteed_true_predicate) {
+  auto conjunction_members = GuaranteeConjunctionMembers(guaranteed_true_predicate);
+  std::unordered_map<FieldRef, Datum, FieldRef::Hash> known_values;
+  RETURN_NOT_OK(ExtractKnownFieldValuesImpl(&conjunction_members, &known_values));
+  return known_values;
+}
+
+Result<Expression> ReplaceFieldsWithKnownValues(
+    const std::unordered_map<FieldRef, Datum, FieldRef::Hash>& known_values,
+    Expression expr) {
+  if (!expr.IsBound()) {
+    return Status::Invalid(
+        "ReplaceFieldsWithKnownValues called on an unbound Expression");
+  }
+
+  return Modify(
+      std::move(expr),
+      [&known_values](Expression expr) -> Result<Expression> {
+        if (auto ref = expr.field_ref()) {
+          auto it = known_values.find(*ref);
+          if (it != known_values.end()) {
+            ARROW_ASSIGN_OR_RAISE(Datum lit,
+                                  compute::Cast(it->second, expr.descr().type));
+            return literal(std::move(lit));
+          }
+        }
+        return expr;
+      },
+      [](Expression expr, ...) { return expr; });
+}
+
+inline bool IsBinaryAssociativeCommutative(const Expression::Call& call) {
+  static std::unordered_set<std::string> binary_associative_commutative{
+      "and",      "or",  "and_kleene",       "or_kleene",  "xor",
+      "multiply", "add", "multiply_checked", "add_checked"};
+
+  auto it = binary_associative_commutative.find(call.function_name);
+  return it != binary_associative_commutative.end();
+}
+
+Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Canonicalize(std::move(expr), &exec_context);
+  }
+
+  // If potentially reconstructing more deeply than a call's immediate arguments
+  // (for example, when reorganizing an associative chain), add expressions to this set to
+  // avoid unnecessary work
+  struct {
+    std::unordered_set<Expression, Expression::Hash> set_;
+
+    bool operator()(const Expression& expr) const {
+      return set_.find(expr) != set_.end();
+    }
+
+    void Add(std::vector<Expression> exprs) {
+      std::move(exprs.begin(), exprs.end(), std::inserter(set_, set_.end()));
+    }
+  } AlreadyCanonicalized;
+
+  return Modify(
+      std::move(expr),
+      [&AlreadyCanonicalized, exec_context](Expression expr) -> Result<Expression> {
+        auto call = expr.call();
+        if (!call) return expr;
+
+        if (AlreadyCanonicalized(expr)) return expr;
+
+        if (IsBinaryAssociativeCommutative(*call)) {
+          struct {
+            int Priority(const Expression& operand) const {
+              // order literals first, starting with nulls

Review comment:
       Perhaps this is not important, but note the reverse convention is used for comparisons...

##########
File path: cpp/src/arrow/compute/kernels/scalar_cast_internal.cc
##########
@@ -149,6 +149,18 @@ void CastNumberToNumberUnsafe(Type::type in_type, Type::type out_type, const Dat
 // ----------------------------------------------------------------------
 
 void UnpackDictionary(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  auto Finish = [&](Result<Datum> result) {
+    if (!result.ok()) {
+      ctx->SetStatus(result.status());
+      return;
+    }
+    *out = *result;
+  };
+
+  if (out->is_scalar()) {
+    return Finish(batch[0].scalar_as<DictionaryScalar>().GetEncodedValue());

Review comment:
       `KERNEL_ASSIGN_OR_RAISE(*out, ctx, batch[0].scalar_as<DictionaryScalar>().GetEncodedValue())` should work

##########
File path: cpp/src/arrow/compute/kernels/util_internal.cc
##########
@@ -57,6 +57,29 @@ PrimitiveArg GetPrimitiveArg(const ArrayData& arr) {
   return arg;
 }
 
+ArrayKernelExec TrivialScalarUnaryAsArraysExec(ArrayKernelExec exec) {
+  return [exec](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (out->is_array()) {
+      return exec(ctx, batch, out);
+    }
+
+    if (!batch[0].scalar()->is_valid) {
+      out->scalar()->is_valid = false;
+      return;
+    }
+
+    Datum array_in, array_out;
+    KERNEL_RETURN_IF_ERROR(

Review comment:
       `KERNEL_ASSIGN_OR_RAISE(array_in, ctx, ...)`?

##########
File path: cpp/src/arrow/compute/kernels/util_internal.cc
##########
@@ -57,6 +57,29 @@ PrimitiveArg GetPrimitiveArg(const ArrayData& arr) {
   return arg;
 }
 
+ArrayKernelExec TrivialScalarUnaryAsArraysExec(ArrayKernelExec exec) {
+  return [exec](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (out->is_array()) {
+      return exec(ctx, batch, out);
+    }
+
+    if (!batch[0].scalar()->is_valid) {
+      out->scalar()->is_valid = false;
+      return;
+    }
+
+    Datum array_in, array_out;
+    KERNEL_RETURN_IF_ERROR(
+        ctx, MakeArrayFromScalar(*batch[0].scalar(), 1).As<Datum>().Value(&array_in));
+    KERNEL_RETURN_IF_ERROR(
+        ctx, MakeArrayFromScalar(*out->scalar(), 1).As<Datum>().Value(&array_out));

Review comment:
       This is just like `MakeArrayOfNull`, right?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();
+
+  compute::KernelContext kernel_context(exec_context);
+  kernel_context.SetState(call->kernel_state.get());
+
+  auto kernel = call->kernel;
+  auto descrs = GetDescriptors(arguments);
+  auto options = call->options.get();
+  RETURN_NOT_OK(executor->Init(&kernel_context, {kernel, descrs, options}));
+
+  auto listener = std::make_shared<compute::detail::DatumAccumulator>();
+  RETURN_NOT_OK(executor->Execute(arguments, listener.get()));
+  return executor->WrapResults(arguments, listener->values());
+}
+
+std::array<std::pair<const Expression&, const Expression&>, 2>
+ArgumentsAndFlippedArguments(const Expression::Call& call) {
+  DCHECK_EQ(call.arguments.size(), 2);
+  return {std::pair<const Expression&, const Expression&>{call.arguments[0],
+                                                          call.arguments[1]},
+          std::pair<const Expression&, const Expression&>{call.arguments[1],
+                                                          call.arguments[0]}};
+}
+
+template <typename BinOp, typename It,
+          typename Out = typename std::iterator_traits<It>::value_type>
+util::optional<Out> FoldLeft(It begin, It end, const BinOp& bin_op) {
+  if (begin == end) return util::nullopt;
+
+  Out folded = std::move(*begin++);
+  while (begin != end) {
+    folded = bin_op(std::move(folded), std::move(*begin++));
+  }
+  return folded;
+}
+
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  if (call.function && call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
+  }
+  return util::nullopt;
+}
+
+bool DefinitelyNotNull(const Expression& expr) {
+  DCHECK(expr.IsBound());
+
+  if (expr.literal()) {
+    return !expr.IsNullLiteral();
+  }
+
+  if (expr.field_ref()) return false;
+
+  auto call = CallNotNull(expr);
+  if (auto null_handling = GetNullHandling(*call)) {
+    if (null_handling == compute::NullHandling::OUTPUT_NOT_NULL) {
+      return true;
+    }
+    if (null_handling == compute::NullHandling::INTERSECTION) {
+      return std::all_of(call->arguments.begin(), call->arguments.end(),
+                         DefinitelyNotNull);
+    }
+  }
+
+  return false;
+}
+
+std::vector<FieldRef> FieldsInExpression(const Expression& expr) {
+  if (auto lit = expr.literal()) return {};
+
+  if (auto ref = expr.field_ref()) {
+    return {*ref};
+  }
+
+  std::vector<FieldRef> fields;
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    auto argument_fields = FieldsInExpression(arg);
+    std::move(argument_fields.begin(), argument_fields.end(), std::back_inserter(fields));
+  }
+  return fields;
+}
+
+Result<Expression> FoldConstants(Expression expr) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [](Expression expr, ...) -> Result<Expression> {
+        auto call = CallNotNull(expr);
+        if (std::all_of(call->arguments.begin(), call->arguments.end(),
+                        [](const Expression& argument) { return argument.literal(); })) {
+          // all arguments are literal; we can evaluate this subexpression *now*
+          static const Datum ignored_input;
+          ARROW_ASSIGN_OR_RAISE(Datum constant,
+                                ExecuteScalarExpression(expr, ignored_input));
+
+          return literal(std::move(constant));
+        }
+
+        // XXX the following should probably be in a registry of passes instead
+        // of inline
+
+        if (GetNullHandling(*call) == compute::NullHandling::INTERSECTION) {
+          // kernels which always produce intersected validity can be resolved
+          // to null *now* if any of their inputs is a null literal
+          for (const auto& argument : call->arguments) {
+            if (argument.IsNullLiteral()) {
+              return argument;
+            }
+          }
+        }
+
+        if (call->function_name == "and_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // true and x == x
+            if (args.first == literal(true)) return args.second;
+
+            // false and x == false
+            if (args.first == literal(false)) return args.first;
+
+            // x and x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        if (call->function_name == "or_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // false or x == x
+            if (args.first == literal(false)) return args.second;
+
+            // true or x == true
+            if (args.first == literal(true)) return args.first;
+
+            // x or x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        return expr;
+      });
+}
+
+inline std::vector<Expression> GuaranteeConjunctionMembers(
+    const Expression& guaranteed_true_predicate) {
+  auto guarantee = guaranteed_true_predicate.call();
+  if (!guarantee || guarantee->function_name != "and_kleene") {
+    return {guaranteed_true_predicate};
+  }
+  return FlattenedAssociativeChain(guaranteed_true_predicate).fringe;
+}
+
+// Conjunction members which are represented in known_values are erased from
+// conjunction_members
+Status ExtractKnownFieldValuesImpl(
+    std::vector<Expression>* conjunction_members,
+    std::unordered_map<FieldRef, Datum, FieldRef::Hash>* known_values) {
+  auto unconsumed_end =
+      std::partition(conjunction_members->begin(), conjunction_members->end(),
+                     [](const Expression& expr) {
+                       // search for an equality conditions between a field and a literal
+                       auto call = expr.call();
+                       if (!call) return true;
+
+                       if (call->function_name == "equal") {
+                         auto ref = call->arguments[0].field_ref();
+                         auto lit = call->arguments[1].literal();
+                         return !(ref && lit);
+                       }
+
+                       return true;
+                     });
+
+  for (auto it = unconsumed_end; it != conjunction_members->end(); ++it) {
+    auto call = CallNotNull(*it);
+
+    auto ref = call->arguments[0].field_ref();
+    auto lit = call->arguments[1].literal();
+
+    auto it_success = known_values->emplace(*ref, *lit);
+    if (it_success.second) continue;
+
+    // A value was already known for ref; check it
+    auto ref_lit = it_success.first;
+    if (*lit != ref_lit->second) {
+      return Status::Invalid("Conflicting guarantees: (", ref->ToString(),
+                             " == ", lit->ToString(), ") vs (", ref->ToString(),
+                             " == ", ref_lit->second.ToString());
+    }
+  }
+
+  conjunction_members->erase(unconsumed_end, conjunction_members->end());
+
+  return Status::OK();
+}
+
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(
+    const Expression& guaranteed_true_predicate) {
+  auto conjunction_members = GuaranteeConjunctionMembers(guaranteed_true_predicate);
+  std::unordered_map<FieldRef, Datum, FieldRef::Hash> known_values;
+  RETURN_NOT_OK(ExtractKnownFieldValuesImpl(&conjunction_members, &known_values));
+  return known_values;
+}
+
+Result<Expression> ReplaceFieldsWithKnownValues(
+    const std::unordered_map<FieldRef, Datum, FieldRef::Hash>& known_values,
+    Expression expr) {
+  if (!expr.IsBound()) {
+    return Status::Invalid(
+        "ReplaceFieldsWithKnownValues called on an unbound Expression");
+  }
+
+  return Modify(
+      std::move(expr),
+      [&known_values](Expression expr) -> Result<Expression> {
+        if (auto ref = expr.field_ref()) {
+          auto it = known_values.find(*ref);
+          if (it != known_values.end()) {
+            ARROW_ASSIGN_OR_RAISE(Datum lit,
+                                  compute::Cast(it->second, expr.descr().type));
+            return literal(std::move(lit));
+          }
+        }
+        return expr;
+      },
+      [](Expression expr, ...) { return expr; });
+}
+
+inline bool IsBinaryAssociativeCommutative(const Expression::Call& call) {
+  static std::unordered_set<std::string> binary_associative_commutative{
+      "and",      "or",  "and_kleene",       "or_kleene",  "xor",
+      "multiply", "add", "multiply_checked", "add_checked"};
+
+  auto it = binary_associative_commutative.find(call.function_name);
+  return it != binary_associative_commutative.end();
+}
+
+Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Canonicalize(std::move(expr), &exec_context);
+  }
+
+  // If potentially reconstructing more deeply than a call's immediate arguments
+  // (for example, when reorganizing an associative chain), add expressions to this set to
+  // avoid unnecessary work
+  struct {
+    std::unordered_set<Expression, Expression::Hash> set_;
+
+    bool operator()(const Expression& expr) const {
+      return set_.find(expr) != set_.end();
+    }
+
+    void Add(std::vector<Expression> exprs) {
+      std::move(exprs.begin(), exprs.end(), std::inserter(set_, set_.end()));
+    }
+  } AlreadyCanonicalized;
+
+  return Modify(
+      std::move(expr),
+      [&AlreadyCanonicalized, exec_context](Expression expr) -> Result<Expression> {
+        auto call = expr.call();
+        if (!call) return expr;
+
+        if (AlreadyCanonicalized(expr)) return expr;
+
+        if (IsBinaryAssociativeCommutative(*call)) {
+          struct {
+            int Priority(const Expression& operand) const {
+              // order literals first, starting with nulls
+              if (operand.IsNullLiteral()) return 0;
+              if (operand.literal()) return 1;
+              return 2;
+            }
+            bool operator()(const Expression& l, const Expression& r) const {
+              return Priority(l) < Priority(r);
+            }
+          } CanonicalOrdering;
+
+          FlattenedAssociativeChain chain(expr);
+          if (chain.was_left_folded &&
+              std::is_sorted(chain.fringe.begin(), chain.fringe.end(),
+                             CanonicalOrdering)) {
+            AlreadyCanonicalized.Add(std::move(chain.exprs));
+            return expr;
+          }
+
+          std::stable_sort(chain.fringe.begin(), chain.fringe.end(), CanonicalOrdering);
+
+          // fold the chain back up
+          auto folded =
+              FoldLeft(chain.fringe.begin(), chain.fringe.end(),
+                       [call, &AlreadyCanonicalized](Expression l, Expression r) {
+                         auto canonicalized_call = *call;
+                         canonicalized_call.arguments = {std::move(l), std::move(r)};
+                         Expression expr(std::move(canonicalized_call));
+                         AlreadyCanonicalized.Add({expr});
+                         return expr;
+                       });
+          return std::move(*folded);
+        }
+
+        if (auto cmp = Comparison::Get(call->function_name)) {
+          if (call->arguments[0].literal() && !call->arguments[1].literal()) {
+            // ensure that literals are on comparisons' RHS
+            auto flipped_call = *call;
+            flipped_call.function_name =
+                Comparison::GetName(Comparison::GetFlipped(*cmp));
+            // look up the flipped kernel
+            // TODO extract a helper for use here and in Bind
+            ARROW_ASSIGN_OR_RAISE(
+                auto function,
+                exec_context->func_registry()->GetFunction(flipped_call.function_name));
+
+            auto descrs = GetDescriptors(flipped_call.arguments);
+            ARROW_ASSIGN_OR_RAISE(flipped_call.kernel, function->DispatchExact(descrs));
+
+            std::swap(flipped_call.arguments[0], flipped_call.arguments[1]);
+            return Expression(std::move(flipped_call));
+          }
+        }
+
+        return expr;
+      },
+      [](Expression expr, ...) { return expr; });
+}
+
+Result<Expression> DirectComparisonSimplification(Expression expr,
+                                                  const Expression::Call& guarantee) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [&guarantee](Expression expr, ...) -> Result<Expression> {
+        auto call = expr.call();
+        if (!call) return expr;
+
+        // Ensure both calls are comparisons with equal LHS and scalar RHS
+        auto cmp = Comparison::Get(expr);
+        auto cmp_guarantee = Comparison::Get(guarantee.function_name);
+        if (!cmp || !cmp_guarantee) return expr;
+
+        if (call->arguments[0] != guarantee.arguments[0]) return expr;
+
+        auto rhs = call->arguments[1].literal();
+        auto guarantee_rhs = guarantee.arguments[1].literal();
+        if (!rhs || !guarantee_rhs) return expr;
+
+        if (!rhs->is_scalar() || !guarantee_rhs->is_scalar()) {
+          return expr;
+        }
+
+        ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_guarantee_rhs,
+                              Comparison::Execute(*rhs, *guarantee_rhs));
+        DCHECK_NE(cmp_rhs_guarantee_rhs, Comparison::NA);
+
+        if (cmp_rhs_guarantee_rhs == Comparison::EQUAL) {
+          // RHS of filter is equal to RHS of guarantee
+
+          if ((*cmp_guarantee & *cmp) == *cmp_guarantee) {
+            // guarantee is a subset of filter, so all data will be included
+            return literal(true);
+          }
+
+          if ((*cmp_guarantee & *cmp) == 0) {
+            // guarantee disjoint with filter, so all data will be excluded
+            return literal(false);
+          }
+
+          return expr;
+        }
+
+        if (*cmp_guarantee & cmp_rhs_guarantee_rhs) {
+          // unusable guarantee
+          return expr;
+        }
+
+        if (*cmp & Comparison::GetFlipped(cmp_rhs_guarantee_rhs)) {
+          // x > 1, x >= 1, x != 1 guaranteed by x >= 3
+          return literal(true);
+        } else {
+          // x < 1, x <= 1, x == 1 unsatisfiable if x >= 3
+          return literal(false);
+        }
+      });
+}
+
+Result<Expression> SimplifyWithGuarantee(Expression expr,
+                                         const Expression& guaranteed_true_predicate) {
+  auto conjunction_members = GuaranteeConjunctionMembers(guaranteed_true_predicate);
+
+  std::unordered_map<FieldRef, Datum, FieldRef::Hash> known_values;
+  RETURN_NOT_OK(ExtractKnownFieldValuesImpl(&conjunction_members, &known_values));
+
+  ARROW_ASSIGN_OR_RAISE(expr,
+                        ReplaceFieldsWithKnownValues(known_values, std::move(expr)));
+
+  auto CanonicalizeAndFoldConstants = [&expr] {
+    ARROW_ASSIGN_OR_RAISE(expr, Canonicalize(std::move(expr)));
+    ARROW_ASSIGN_OR_RAISE(expr, FoldConstants(std::move(expr)));
+    return Status::OK();
+  };
+  RETURN_NOT_OK(CanonicalizeAndFoldConstants());
+
+  for (const auto& guarantee : conjunction_members) {

Review comment:
       Instead of looping on conjunction members, have you tried to match all members at once in the post-visit callback in `DirectComparisonSimplification`?

##########
File path: cpp/src/arrow/dataset/partition.h
##########
@@ -294,5 +285,22 @@ class ARROW_DS_EXPORT PartitioningOrFactory {
   std::shared_ptr<Partitioning> partitioning_;
 };
 
+/// \brief Assemble lists of indices of identical rows.
+///
+/// \param[in] by A StructArray whose columns will be used as grouping criteria.
+/// \return A StructArray mapping unique rows (in field "values", represented as a
+///         StructArray with the same fields as `by`) to lists of indices where
+///         that row appears (in field "groupings").

Review comment:
       So the return type is `struct{values: <by.type>, groupings: list[int64]}`?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally

Review comment:
       Of course, this may be deferred to a later JIRA. But should add a TODO and open the JIRA at least.

##########
File path: cpp/src/arrow/dataset/partition.h
##########
@@ -63,13 +63,13 @@ class ARROW_DS_EXPORT Partitioning {
   /// produce sub-batches which satisfy mutually exclusive Expressions.
   struct PartitionedBatches {
     RecordBatchVector batches;
-    ExpressionVector expressions;
+    std::vector<Expression> expressions;

Review comment:
       We could keep `ExpressionVector`...

##########
File path: cpp/src/arrow/dataset/scanner.h
##########
@@ -62,10 +63,7 @@ class ARROW_DS_EXPORT ScanOptions {
   std::shared_ptr<ScanOptions> ReplaceSchema(std::shared_ptr<Schema> schema) const;
 
   // Filter
-  std::shared_ptr<Expression> filter = scalar(true);
-
-  // Evaluator for Filter
-  std::shared_ptr<ExpressionEvaluator> evaluator;
+  Expression filter2 = literal(true);

Review comment:
       Why `filter2`? Is it just to help with refactoring? (will it be renamed back to `filter`?)

##########
File path: cpp/src/arrow/dataset/partition_test.cc
##########
@@ -21,52 +21,51 @@
 #include <gtest/gtest.h>
 
 #include <cstdint>
-#include <map>
 #include <memory>
 #include <regex>
 #include <string>
 #include <vector>
 
-#include "arrow/dataset/file_base.h"
 #include "arrow/dataset/scanner_internal.h"
 #include "arrow/dataset/test_util.h"
-#include "arrow/filesystem/localfs.h"
 #include "arrow/filesystem/path_util.h"
 #include "arrow/status.h"
 #include "arrow/testing/gtest_util.h"
-#include "arrow/util/io_util.h"
 
 namespace arrow {
 using internal::checked_pointer_cast;
 
 namespace dataset {
 
-using E = TestExpression;
-
 class TestPartitioning : public ::testing::Test {
  public:
   void AssertParseError(const std::string& path) {
     ASSERT_RAISES(Invalid, partitioning_->Parse(path));
   }
 
-  void AssertParse(const std::string& path, E expected) {
+  void AssertParse(const std::string& path, Expression expected) {
     ASSERT_OK_AND_ASSIGN(auto parsed, partitioning_->Parse(path));
-    ASSERT_EQ(E{parsed}, expected);
+    ASSERT_EQ(parsed, expected);
   }
 
   template <StatusCode code = StatusCode::Invalid>
-  void AssertFormatError(E expr) {
-    ASSERT_EQ(partitioning_->Format(*expr.expression).status().code(), code);
+  void AssertFormatError(Expression expr) {
+    ASSERT_EQ(partitioning_->Format(expr).status().code(), code);
   }
 
-  void AssertFormat(E expr, const std::string& expected) {
-    ASSERT_OK_AND_ASSIGN(auto formatted, partitioning_->Format(*expr.expression));
+  void AssertFormat(Expression expr, const std::string& expected) {
+    // formatted partition expressions are bound to the schema of the dataset being
+    // written
+    ASSERT_OK_AND_ASSIGN(auto formatted, partitioning_->Format(expr));
     ASSERT_EQ(formatted, expected);
 
     // ensure the formatted path round trips the relevant components of the partition
     // expression: roundtripped should be a subset of expr
-    ASSERT_OK_AND_ASSIGN(auto roundtripped, partitioning_->Parse(formatted));
-    ASSERT_EQ(E{roundtripped->Assume(*expr.expression)}, E{scalar(true)});
+    ASSERT_OK_AND_ASSIGN(Expression roundtripped, partitioning_->Parse(formatted));
+
+    ASSERT_OK_AND_ASSIGN(roundtripped, roundtripped.Bind(*written_schema_));

Review comment:
       If we format `roundtripped` again, it should produce the same string, right?

##########
File path: cpp/src/arrow/dataset/partition.cc
##########
@@ -573,5 +530,192 @@ Result<std::shared_ptr<Schema>> PartitioningOrFactory::GetOrInferSchema(
   return factory()->Inspect(paths);
 }
 
+// Transform an array of counts to offsets which will divide a ListArray
+// into an equal number of slices with corresponding lengths.
+inline Result<std::shared_ptr<Array>> CountsToOffsets(
+    std::shared_ptr<Int64Array> counts) {
+  Int32Builder offset_builder;

Review comment:
       Not sure this is performance-critical, but if it is, note that `Int32Builder` will pointlessly build up a null bitmap bit by bit.

##########
File path: cpp/src/arrow/dataset/expression.h
##########
@@ -0,0 +1,234 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This API is EXPERIMENTAL.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/cast.h"
+#include "arrow/dataset/type_fwd.h"
+#include "arrow/dataset/visibility.h"
+#include "arrow/datum.h"
+#include "arrow/result.h"
+#include "arrow/scalar.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/variant.h"
+
+namespace arrow {
+namespace dataset {
+
+/// An unbound expression which maps a single Datum to another Datum.
+/// An expression is one of
+/// - A literal Datum.
+/// - A reference to a single (potentially nested) field of the input Datum.
+/// - A call to a compute function, with arguments specified by other Expressions.
+class ARROW_DS_EXPORT Expression {
+ public:
+  struct Call {
+    std::string function_name;
+    std::vector<Expression> arguments;
+    std::shared_ptr<compute::FunctionOptions> options;
+
+    // post-Bind properties:
+    const compute::Kernel* kernel = NULLPTR;
+    std::shared_ptr<compute::Function> function;
+    std::shared_ptr<compute::KernelState> kernel_state;
+    ValueDescr descr;
+  };
+
+  std::string ToString() const;
+  bool Equals(const Expression& other) const;
+  size_t hash() const;
+  struct Hash {
+    size_t operator()(const Expression& expr) const { return expr.hash(); }
+  };
+
+  /// Bind this expression to the given input type, looking up Kernels and field types.
+  /// Some expression simplification may be performed and implicit casts will be inserted.
+  /// Any state necessary for execution will be initialized and returned.
+  Result<Expression> Bind(ValueDescr in, compute::ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(const Schema& in_schema, compute::ExecContext* = NULLPTR) const;
+
+  // XXX someday
+  // Clone all KernelState in this bound expression. If any function referenced by this
+  // expression has mutable KernelState, it is not safe to execute or apply simplification
+  // passes to it (or copies of it!) from multiple threads. Cloning state produces new
+  // KernelStates where necessary to ensure that Expressions may be manipulated safely
+  // on multiple threads.
+  // Result<ExpressionState> CloneState() const;
+  // Status SetState(ExpressionState);
+
+  /// Return true if all an expression's field references have explicit ValueDescr and all
+  /// of its functions' kernels are looked up.
+  bool IsBound() const;
+
+  /// Return true if this expression is composed only of Scalar literals, field
+  /// references, and calls to ScalarFunctions.
+  bool IsScalarExpression() const;
+
+  /// Return true if this expression is literal and entirely null.
+  bool IsNullLiteral() const;
+
+  /// Return true if this expression could evaluate to true.
+  bool IsSatisfiable() const;
+
+  // XXX someday
+  // Result<PipelineGraph> GetPipelines();
+
+  const Call* call() const;
+  const Datum* literal() const;
+  const FieldRef* field_ref() const;
+
+  ValueDescr descr() const;
+  // XXX someday
+  // NullGeneralization::type nullable() const;
+
+  struct Parameter {
+    FieldRef ref;
+    ValueDescr descr;
+  };
+
+  Expression() = default;
+  explicit Expression(Call call);
+  explicit Expression(Datum literal);
+  explicit Expression(Parameter parameter);
+
+ private:
+  using Impl = util::Variant<Datum, Parameter, Call>;
+  std::shared_ptr<Impl> impl_;
+
+  ARROW_EXPORT friend bool Identical(const Expression& l, const Expression& r);
+
+  ARROW_EXPORT friend void PrintTo(const Expression&, std::ostream*);
+};
+
+inline bool operator==(const Expression& l, const Expression& r) { return l.Equals(r); }
+inline bool operator!=(const Expression& l, const Expression& r) { return !l.Equals(r); }
+
+// Factories
+
+ARROW_DS_EXPORT
+Expression literal(Datum lit);
+
+template <typename Arg>
+Expression literal(Arg&& arg) {
+  return literal(Datum(std::forward<Arg>(arg)));
+}
+
+ARROW_DS_EXPORT
+Expression field_ref(FieldRef ref);
+
+ARROW_DS_EXPORT
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options = NULLPTR);
+
+template <typename Options, typename = typename std::enable_if<std::is_base_of<
+                                compute::FunctionOptions, Options>::value>::type>
+Expression call(std::string function, std::vector<Expression> arguments,
+                Options options) {
+  return call(std::move(function), std::move(arguments),
+              std::make_shared<Options>(std::move(options)));
+}
+
+ARROW_DS_EXPORT
+std::vector<FieldRef> FieldsInExpression(const Expression&);
+
+ARROW_DS_EXPORT
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(
+    const Expression& guaranteed_true_predicate);
+
+/// \defgroup expression-passes Functions for modification of Expressions
+///
+/// @{
+///
+/// These operate on bound expressions.
+
+/// Weak canonicalization which establishes guarantees for subsequent passes. Even
+/// equivalent Expressions may result in different canonicalized expressions.
+/// TODO this could be a strong canonicalization
+ARROW_DS_EXPORT
+Result<Expression> Canonicalize(Expression, compute::ExecContext* = NULLPTR);
+
+/// Simplify Expressions based on literal arguments (for example, add(null, x) will always
+/// be null so replace the call with a null literal). Includes early evaluation of all
+/// calls whose arguments are entirely literal.
+ARROW_DS_EXPORT
+Result<Expression> FoldConstants(Expression);
+
+ARROW_DS_EXPORT
+Result<Expression> ReplaceFieldsWithKnownValues(
+    const std::unordered_map<FieldRef, Datum, FieldRef::Hash>& known_values, Expression);
+
+/// Simplify an expression by replacing subexpressions based on a guarantee:
+/// a boolean expression which is guaranteed to evaluate to `true`. For example, this is
+/// used to remove redundant function calls from a filter expression or to replace a
+/// reference to a constant-value field with a literal.
+ARROW_DS_EXPORT
+Result<Expression> SimplifyWithGuarantee(Expression,
+                                         const Expression& guaranteed_true_predicate);
+
+/// @}
+
+// Execution
+
+/// Execute a scalar expression against the provided state and input Datum. This

Review comment:
       Hmm... I assume the input Datum should be a record batch or table, so that field refs can be looked up?

##########
File path: cpp/src/arrow/dataset/expression.h
##########
@@ -0,0 +1,234 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This API is EXPERIMENTAL.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/cast.h"
+#include "arrow/dataset/type_fwd.h"
+#include "arrow/dataset/visibility.h"
+#include "arrow/datum.h"
+#include "arrow/result.h"
+#include "arrow/scalar.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/variant.h"
+
+namespace arrow {
+namespace dataset {
+
+/// An unbound expression which maps a single Datum to another Datum.
+/// An expression is one of
+/// - A literal Datum.
+/// - A reference to a single (potentially nested) field of the input Datum.
+/// - A call to a compute function, with arguments specified by other Expressions.
+class ARROW_DS_EXPORT Expression {
+ public:
+  struct Call {
+    std::string function_name;
+    std::vector<Expression> arguments;
+    std::shared_ptr<compute::FunctionOptions> options;
+
+    // post-Bind properties:
+    const compute::Kernel* kernel = NULLPTR;
+    std::shared_ptr<compute::Function> function;
+    std::shared_ptr<compute::KernelState> kernel_state;
+    ValueDescr descr;
+  };
+
+  std::string ToString() const;
+  bool Equals(const Expression& other) const;
+  size_t hash() const;
+  struct Hash {
+    size_t operator()(const Expression& expr) const { return expr.hash(); }
+  };
+
+  /// Bind this expression to the given input type, looking up Kernels and field types.
+  /// Some expression simplification may be performed and implicit casts will be inserted.
+  /// Any state necessary for execution will be initialized and returned.
+  Result<Expression> Bind(ValueDescr in, compute::ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(const Schema& in_schema, compute::ExecContext* = NULLPTR) const;
+
+  // XXX someday
+  // Clone all KernelState in this bound expression. If any function referenced by this
+  // expression has mutable KernelState, it is not safe to execute or apply simplification
+  // passes to it (or copies of it!) from multiple threads. Cloning state produces new
+  // KernelStates where necessary to ensure that Expressions may be manipulated safely
+  // on multiple threads.
+  // Result<ExpressionState> CloneState() const;
+  // Status SetState(ExpressionState);
+
+  /// Return true if all an expression's field references have explicit ValueDescr and all
+  /// of its functions' kernels are looked up.
+  bool IsBound() const;
+
+  /// Return true if this expression is composed only of Scalar literals, field
+  /// references, and calls to ScalarFunctions.
+  bool IsScalarExpression() const;
+
+  /// Return true if this expression is literal and entirely null.
+  bool IsNullLiteral() const;
+
+  /// Return true if this expression could evaluate to true.
+  bool IsSatisfiable() const;
+
+  // XXX someday
+  // Result<PipelineGraph> GetPipelines();
+
+  const Call* call() const;
+  const Datum* literal() const;
+  const FieldRef* field_ref() const;
+
+  ValueDescr descr() const;
+  // XXX someday
+  // NullGeneralization::type nullable() const;
+
+  struct Parameter {
+    FieldRef ref;
+    ValueDescr descr;
+  };
+
+  Expression() = default;
+  explicit Expression(Call call);
+  explicit Expression(Datum literal);
+  explicit Expression(Parameter parameter);
+
+ private:
+  using Impl = util::Variant<Datum, Parameter, Call>;
+  std::shared_ptr<Impl> impl_;
+
+  ARROW_EXPORT friend bool Identical(const Expression& l, const Expression& r);
+
+  ARROW_EXPORT friend void PrintTo(const Expression&, std::ostream*);
+};
+
+inline bool operator==(const Expression& l, const Expression& r) { return l.Equals(r); }
+inline bool operator!=(const Expression& l, const Expression& r) { return !l.Equals(r); }
+
+// Factories
+
+ARROW_DS_EXPORT
+Expression literal(Datum lit);
+
+template <typename Arg>
+Expression literal(Arg&& arg) {
+  return literal(Datum(std::forward<Arg>(arg)));
+}
+
+ARROW_DS_EXPORT
+Expression field_ref(FieldRef ref);
+
+ARROW_DS_EXPORT
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options = NULLPTR);
+
+template <typename Options, typename = typename std::enable_if<std::is_base_of<
+                                compute::FunctionOptions, Options>::value>::type>
+Expression call(std::string function, std::vector<Expression> arguments,
+                Options options) {
+  return call(std::move(function), std::move(arguments),
+              std::make_shared<Options>(std::move(options)));
+}
+
+ARROW_DS_EXPORT
+std::vector<FieldRef> FieldsInExpression(const Expression&);
+
+ARROW_DS_EXPORT
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(
+    const Expression& guaranteed_true_predicate);
+
+/// \defgroup expression-passes Functions for modification of Expressions
+///
+/// @{
+///
+/// These operate on bound expressions.
+
+/// Weak canonicalization which establishes guarantees for subsequent passes. Even
+/// equivalent Expressions may result in different canonicalized expressions.
+/// TODO this could be a strong canonicalization
+ARROW_DS_EXPORT
+Result<Expression> Canonicalize(Expression, compute::ExecContext* = NULLPTR);
+
+/// Simplify Expressions based on literal arguments (for example, add(null, x) will always
+/// be null so replace the call with a null literal). Includes early evaluation of all
+/// calls whose arguments are entirely literal.

Review comment:
       At some point we'll want to be able to flag compute functions as "impure", for example a now() or random() function.

##########
File path: cpp/src/arrow/dataset/scanner_test.cc
##########
@@ -184,14 +183,26 @@ TEST_F(TestScannerBuilder, TestProject) {
 TEST_F(TestScannerBuilder, TestFilter) {
   ScannerBuilder builder(dataset_, ctx_);
 
-  ASSERT_OK(builder.Filter(scalar(true)));
-  ASSERT_OK(builder.Filter("i64"_ == int64_t(10)));
-  ASSERT_OK(builder.Filter("i64"_ == int64_t(10) || "b"_ == true));
-
-  ASSERT_RAISES(TypeError, builder.Filter("i64"_ == int32_t(10)));
-  ASSERT_RAISES(Invalid, builder.Filter("not_a_column"_ == true));
-  ASSERT_RAISES(Invalid,
-                builder.Filter("i64"_ == int64_t(10) || "not_a_column"_ == true));
+  ASSERT_OK(builder.Filter(literal(true)));
+  ASSERT_OK(builder.Filter(call("equal", {field_ref("i64"), literal<int64_t>(10)})));
+  ASSERT_OK(builder.Filter(
+      call("or_kleene", {

Review comment:
       Is `call("or_kleene", {Expression, Expression})` the same as `or_(Expression, Expression)`?

##########
File path: cpp/src/arrow/dataset/partition.cc
##########
@@ -573,5 +530,192 @@ Result<std::shared_ptr<Schema>> PartitioningOrFactory::GetOrInferSchema(
   return factory()->Inspect(paths);
 }
 
+// Transform an array of counts to offsets which will divide a ListArray
+// into an equal number of slices with corresponding lengths.
+inline Result<std::shared_ptr<Array>> CountsToOffsets(
+    std::shared_ptr<Int64Array> counts) {
+  Int32Builder offset_builder;
+  RETURN_NOT_OK(offset_builder.Resize(counts->length() + 1));
+  offset_builder.UnsafeAppend(0);
+
+  for (int64_t i = 0; i < counts->length(); ++i) {
+    DCHECK_NE(counts->Value(i), 0);
+    auto next_offset = static_cast<int32_t>(offset_builder[i] + counts->Value(i));
+    offset_builder.UnsafeAppend(next_offset);
+  }
+
+  std::shared_ptr<Array> offsets;
+  RETURN_NOT_OK(offset_builder.Finish(&offsets));
+  return offsets;
+}
+
+// Helper for simultaneous dictionary encoding of multiple arrays.
+//
+// The fused dictionary is the Cartesian product of the individual dictionaries.
+// For example given two arrays A, B where A has unique values ["ex", "why"]
+// and B has unique values [0, 1] the fused dictionary is the set of tuples
+// [["ex", 0], ["ex", 1], ["why", 0], ["ex", 1]].
+//
+// TODO(bkietz) this capability belongs in an Action of the hash kernels, where
+// it can be used to group aggregates without materializing a grouped batch.
+// For the purposes of writing we need the materialized grouped batch anyway
+// since no Writers accept a selection vector.
+class StructDictionary {
+ public:
+  struct Encoded {
+    std::shared_ptr<Int32Array> indices;
+    std::shared_ptr<StructDictionary> dictionary;
+  };
+
+  static Result<Encoded> Encode(const ArrayVector& columns) {
+    Encoded out{nullptr, std::make_shared<StructDictionary>()};
+
+    for (const auto& column : columns) {
+      if (column->null_count() != 0) {
+        return Status::NotImplemented("Grouping on a field with nulls");
+      }
+
+      RETURN_NOT_OK(out.dictionary->AddOne(column, &out.indices));
+    }
+
+    return out;
+  }
+
+  Result<std::shared_ptr<StructArray>> Decode(std::shared_ptr<Int32Array> fused_indices,
+                                              FieldVector fields) {
+    std::vector<Int32Builder> builders(dictionaries_.size());
+    for (Int32Builder& b : builders) {
+      RETURN_NOT_OK(b.Resize(fused_indices->length()));
+    }
+
+    std::vector<int32_t> codes(dictionaries_.size());
+    for (int64_t i = 0; i < fused_indices->length(); ++i) {
+      Expand(fused_indices->Value(i), codes.data());
+
+      auto builder_it = builders.begin();
+      for (int32_t index : codes) {
+        builder_it++->UnsafeAppend(index);
+      }
+    }
+
+    ArrayVector columns(dictionaries_.size());
+    for (size_t i = 0; i < dictionaries_.size(); ++i) {
+      std::shared_ptr<ArrayData> indices;
+      RETURN_NOT_OK(builders[i].FinishInternal(&indices));
+
+      ARROW_ASSIGN_OR_RAISE(Datum column, compute::Take(dictionaries_[i], indices));
+      columns[i] = column.make_array();
+    }
+
+    return StructArray::Make(std::move(columns), std::move(fields));
+  }
+
+ private:
+  Status AddOne(Datum column, std::shared_ptr<Int32Array>* fused_indices) {
+    ArrayData* encoded;
+    if (column.type()->id() != Type::DICTIONARY) {
+      ARROW_ASSIGN_OR_RAISE(column, compute::DictionaryEncode(column));
+    }
+    encoded = column.mutable_array();
+
+    auto indices =
+        std::make_shared<Int32Array>(encoded->length, std::move(encoded->buffers[1]));

Review comment:
       Be careful, you might have got a dictionary array with a non-int32 index type.

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{
+        {"equal", EQUAL},     {"not_equal", NOT_EQUAL},
+        {"less", LESS},       {"less_equal", LESS_EQUAL},
+        {"greater", GREATER}, {"greater_equal", GREATER_EQUAL},
+    };
+
+    auto it = flipped_comparisons.find(function);
+    return it != flipped_comparisons.end() ? &it->second : nullptr;
+  }
+
+  static const type* Get(const Expression& expr) {
+    if (auto call = expr.call()) {
+      return Comparison::Get(call->function_name);
+    }
+    return nullptr;
+  }
+
+  // Execute a simple Comparison between scalars, casting the RHS if types disagree
+  static Result<type> Execute(Datum l, Datum r) {
+    if (!l.is_scalar() || !r.is_scalar()) {
+      return Status::Invalid("Cannot Execute Comparison on non-scalars");
+    }
+
+    if (!l.type()->Equals(r.type())) {
+      ARROW_ASSIGN_OR_RAISE(r, compute::Cast(r, l.type()));

Review comment:
       This may cast to the narrowest type if e.g. `l` is int8 and `r` is int16. Then, due to truncation, the comparison can produce an incorrect result.

##########
File path: cpp/src/arrow/dataset/partition_test.cc
##########
@@ -99,38 +98,57 @@ class TestPartitioning : public ::testing::Test {
 
   std::shared_ptr<Partitioning> partitioning_;
   std::shared_ptr<PartitioningFactory> factory_;
+  std::shared_ptr<Schema> written_schema_;
 };
 
 TEST_F(TestPartitioning, DirectoryPartitioning) {
   partitioning_ = std::make_shared<DirectoryPartitioning>(
       schema({field("alpha", int32()), field("beta", utf8())}));
 
-  AssertParse("/0/hello", "alpha"_ == int32_t(0) and "beta"_ == "hello");
-  AssertParse("/3", "alpha"_ == int32_t(3));
-  AssertParseError("/world/0");   // reversed order
-  AssertParseError("/0.0/foo");   // invalid alpha
-  AssertParseError("/3.25");      // invalid alpha with missing beta
-  AssertParse("", scalar(true));  // no segments to parse
+  AssertParse("/0/hello", and_(equal(field_ref("alpha"), literal(0)),
+                               equal(field_ref("beta"), literal("hello"))));
+  AssertParse("/3", equal(field_ref("alpha"), literal(3)));
+  AssertParseError("/world/0");    // reversed order
+  AssertParseError("/0.0/foo");    // invalid alpha
+  AssertParseError("/3.25");       // invalid alpha with missing beta
+  AssertParse("", literal(true));  // no segments to parse
 
   // gotcha someday:
-  AssertParse("/0/dat.parquet", "alpha"_ == int32_t(0) and "beta"_ == "dat.parquet");
+  AssertParse("/0/dat.parquet", and_(equal(field_ref("alpha"), literal(0)),
+                                     equal(field_ref("beta"), literal("dat.parquet"))));
 
-  AssertParse("/0/foo/ignored=2341", "alpha"_ == int32_t(0) and "beta"_ == "foo");
+  AssertParse("/0/foo/ignored=2341", and_(equal(field_ref("alpha"), literal(0)),
+                                          equal(field_ref("beta"), literal("foo"))));
 }
 
 TEST_F(TestPartitioning, DirectoryPartitioningFormat) {
   partitioning_ = std::make_shared<DirectoryPartitioning>(
       schema({field("alpha", int32()), field("beta", utf8())}));
 
-  AssertFormat("alpha"_ == int32_t(0) and "beta"_ == "hello", "0/hello");
-  AssertFormat("beta"_ == "hello" and "alpha"_ == int32_t(0), "0/hello");
-  AssertFormat("alpha"_ == int32_t(0), "0");
-  AssertFormatError("beta"_ == "hello");
-  AssertFormat(scalar(true), "");
+  written_schema_ = partitioning_->schema();
 
-  AssertFormatError<StatusCode::TypeError>("alpha"_ == 0.0 and "beta"_ == "hello");
-  AssertFormat("gamma"_ == "yo" and "alpha"_ == int32_t(0) and "beta"_ == "hello",
+  AssertFormat(and_(equal(field_ref("alpha"), literal(0)),
+                    equal(field_ref("beta"), literal("hello"))),
+               "0/hello");
+  AssertFormat(and_(equal(field_ref("beta"), literal("hello")),
+                    equal(field_ref("alpha"), literal(0))),
+               "0/hello");
+  AssertFormat(equal(field_ref("alpha"), literal(0)), "0");
+  AssertFormatError(equal(field_ref("beta"), literal("hello")));
+  AssertFormat(literal(true), "");
+
+  ASSERT_OK_AND_ASSIGN(written_schema_,
+                       written_schema_->AddField(0, field("gamma", utf8())));
+  AssertFormat(and_({equal(field_ref("gamma"), literal("yo")),
+                     equal(field_ref("alpha"), literal(0)),
+                     equal(field_ref("beta"), literal("hello"))}),
                "0/hello");
+
+  // written_schema_ is incompatible with partitioning_'s schema

Review comment:
       I'm not sure I understand this test. What does `written_schema_` correspond to? The concrete schema of a file (e.g. Parquet) present in the dataset?

##########
File path: cpp/src/arrow/result.h
##########
@@ -401,6 +401,28 @@ class ARROW_MUST_USE_TYPE Result : public util::EqualityComparable<Result<T>> {
     return std::forward<M>(m)(ValueUnsafe());
   }
 
+  /// Cast the internally stored value to produce a new result or propagate the stored
+  /// error.
+  template <typename U, typename E = typename std::enable_if<
+                            std::is_constructible<U, T>::value>::type>
+  Result<U> As() && {

Review comment:
       +1

##########
File path: cpp/src/arrow/type.h
##########
@@ -1423,10 +1426,14 @@ class ARROW_EXPORT FieldPath {
   Result<std::shared_ptr<Array>> Get(const RecordBatch& batch) const;
   Result<std::shared_ptr<ChunkedArray>> Get(const Table& table) const;
 
-  /// \brief Retrieve the referenced child Array from an Array or ChunkedArray
+  /// \brief Retrieve the referenced child from an Array, ArrayData, or ChunkedArray
   Result<std::shared_ptr<Array>> Get(const Array& array) const;
+  Result<std::shared_ptr<ArrayData>> Get(const ArrayData& data) const;
   Result<std::shared_ptr<ChunkedArray>> Get(const ChunkedArray& array) const;
 
+  /// \brief Retrieve the reference child from a Datum

Review comment:
       "referenced"?

##########
File path: cpp/src/arrow/type.h
##########
@@ -1423,10 +1426,14 @@ class ARROW_EXPORT FieldPath {
   Result<std::shared_ptr<Array>> Get(const RecordBatch& batch) const;
   Result<std::shared_ptr<ChunkedArray>> Get(const Table& table) const;
 
-  /// \brief Retrieve the referenced child Array from an Array or ChunkedArray
+  /// \brief Retrieve the referenced child from an Array, ArrayData, or ChunkedArray
   Result<std::shared_ptr<Array>> Get(const Array& array) const;
+  Result<std::shared_ptr<ArrayData>> Get(const ArrayData& data) const;
   Result<std::shared_ptr<ChunkedArray>> Get(const ChunkedArray& array) const;
 
+  /// \brief Retrieve the reference child from a Datum
+  Result<Datum> Get(const Datum& datum) const;

Review comment:
       I haven't seen the implementation for this overload. Did I overlook something?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{

Review comment:
       Nit, but these don't look "flipped" to me?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));

Review comment:
       Should we add `std::shared_ptr<Schema> Datum::schema() const`?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{
+        {"equal", EQUAL},     {"not_equal", NOT_EQUAL},
+        {"less", LESS},       {"less_equal", LESS_EQUAL},
+        {"greater", GREATER}, {"greater_equal", GREATER_EQUAL},
+    };
+
+    auto it = flipped_comparisons.find(function);
+    return it != flipped_comparisons.end() ? &it->second : nullptr;
+  }
+
+  static const type* Get(const Expression& expr) {
+    if (auto call = expr.call()) {
+      return Comparison::Get(call->function_name);
+    }
+    return nullptr;
+  }
+
+  // Execute a simple Comparison between scalars, casting the RHS if types disagree

Review comment:
       FTR, what does this return if both inputs are null?

##########
File path: cpp/src/arrow/dataset/scanner_test.cc
##########
@@ -184,14 +183,26 @@ TEST_F(TestScannerBuilder, TestProject) {
 TEST_F(TestScannerBuilder, TestFilter) {
   ScannerBuilder builder(dataset_, ctx_);
 
-  ASSERT_OK(builder.Filter(scalar(true)));
-  ASSERT_OK(builder.Filter("i64"_ == int64_t(10)));
-  ASSERT_OK(builder.Filter("i64"_ == int64_t(10) || "b"_ == true));
-
-  ASSERT_RAISES(TypeError, builder.Filter("i64"_ == int32_t(10)));
-  ASSERT_RAISES(Invalid, builder.Filter("not_a_column"_ == true));
-  ASSERT_RAISES(Invalid,
-                builder.Filter("i64"_ == int64_t(10) || "not_a_column"_ == true));
+  ASSERT_OK(builder.Filter(literal(true)));
+  ASSERT_OK(builder.Filter(call("equal", {field_ref("i64"), literal<int64_t>(10)})));

Review comment:
       Is `call("equal", {Expression, Expression})` the same as `equal(Expression, Expression)`?

##########
File path: cpp/src/arrow/dataset/expression.h
##########
@@ -0,0 +1,234 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This API is EXPERIMENTAL.
+
+#pragma once
+
+#include <functional>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+#include <vector>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/cast.h"
+#include "arrow/dataset/type_fwd.h"
+#include "arrow/dataset/visibility.h"
+#include "arrow/datum.h"
+#include "arrow/result.h"
+#include "arrow/scalar.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/variant.h"
+
+namespace arrow {
+namespace dataset {
+
+/// An unbound expression which maps a single Datum to another Datum.
+/// An expression is one of
+/// - A literal Datum.
+/// - A reference to a single (potentially nested) field of the input Datum.
+/// - A call to a compute function, with arguments specified by other Expressions.
+class ARROW_DS_EXPORT Expression {
+ public:
+  struct Call {
+    std::string function_name;
+    std::vector<Expression> arguments;
+    std::shared_ptr<compute::FunctionOptions> options;
+
+    // post-Bind properties:
+    const compute::Kernel* kernel = NULLPTR;
+    std::shared_ptr<compute::Function> function;
+    std::shared_ptr<compute::KernelState> kernel_state;
+    ValueDescr descr;
+  };
+
+  std::string ToString() const;
+  bool Equals(const Expression& other) const;
+  size_t hash() const;
+  struct Hash {
+    size_t operator()(const Expression& expr) const { return expr.hash(); }
+  };
+
+  /// Bind this expression to the given input type, looking up Kernels and field types.
+  /// Some expression simplification may be performed and implicit casts will be inserted.
+  /// Any state necessary for execution will be initialized and returned.
+  Result<Expression> Bind(ValueDescr in, compute::ExecContext* = NULLPTR) const;
+  Result<Expression> Bind(const Schema& in_schema, compute::ExecContext* = NULLPTR) const;
+
+  // XXX someday
+  // Clone all KernelState in this bound expression. If any function referenced by this
+  // expression has mutable KernelState, it is not safe to execute or apply simplification
+  // passes to it (or copies of it!) from multiple threads. Cloning state produces new
+  // KernelStates where necessary to ensure that Expressions may be manipulated safely
+  // on multiple threads.
+  // Result<ExpressionState> CloneState() const;
+  // Status SetState(ExpressionState);
+
+  /// Return true if all an expression's field references have explicit ValueDescr and all
+  /// of its functions' kernels are looked up.
+  bool IsBound() const;
+
+  /// Return true if this expression is composed only of Scalar literals, field
+  /// references, and calls to ScalarFunctions.
+  bool IsScalarExpression() const;
+
+  /// Return true if this expression is literal and entirely null.
+  bool IsNullLiteral() const;
+
+  /// Return true if this expression could evaluate to true.
+  bool IsSatisfiable() const;
+
+  // XXX someday
+  // Result<PipelineGraph> GetPipelines();
+
+  const Call* call() const;
+  const Datum* literal() const;
+  const FieldRef* field_ref() const;
+
+  ValueDescr descr() const;
+  // XXX someday
+  // NullGeneralization::type nullable() const;
+
+  struct Parameter {
+    FieldRef ref;
+    ValueDescr descr;
+  };
+
+  Expression() = default;
+  explicit Expression(Call call);
+  explicit Expression(Datum literal);
+  explicit Expression(Parameter parameter);
+
+ private:
+  using Impl = util::Variant<Datum, Parameter, Call>;
+  std::shared_ptr<Impl> impl_;
+
+  ARROW_EXPORT friend bool Identical(const Expression& l, const Expression& r);
+
+  ARROW_EXPORT friend void PrintTo(const Expression&, std::ostream*);
+};
+
+inline bool operator==(const Expression& l, const Expression& r) { return l.Equals(r); }
+inline bool operator!=(const Expression& l, const Expression& r) { return !l.Equals(r); }
+
+// Factories
+
+ARROW_DS_EXPORT
+Expression literal(Datum lit);
+
+template <typename Arg>
+Expression literal(Arg&& arg) {
+  return literal(Datum(std::forward<Arg>(arg)));
+}
+
+ARROW_DS_EXPORT
+Expression field_ref(FieldRef ref);
+
+ARROW_DS_EXPORT
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options = NULLPTR);
+
+template <typename Options, typename = typename std::enable_if<std::is_base_of<
+                                compute::FunctionOptions, Options>::value>::type>
+Expression call(std::string function, std::vector<Expression> arguments,
+                Options options) {
+  return call(std::move(function), std::move(arguments),
+              std::make_shared<Options>(std::move(options)));
+}
+
+ARROW_DS_EXPORT
+std::vector<FieldRef> FieldsInExpression(const Expression&);

Review comment:
       Add a docstring? Especially, is this deep or shallow?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{
+        {"equal", EQUAL},     {"not_equal", NOT_EQUAL},
+        {"less", LESS},       {"less_equal", LESS_EQUAL},
+        {"greater", GREATER}, {"greater_equal", GREATER_EQUAL},
+    };
+
+    auto it = flipped_comparisons.find(function);
+    return it != flipped_comparisons.end() ? &it->second : nullptr;
+  }
+
+  static const type* Get(const Expression& expr) {
+    if (auto call = expr.call()) {
+      return Comparison::Get(call->function_name);
+    }
+    return nullptr;
+  }
+
+  // Execute a simple Comparison between scalars, casting the RHS if types disagree
+  static Result<type> Execute(Datum l, Datum r) {
+    if (!l.is_scalar() || !r.is_scalar()) {
+      return Status::Invalid("Cannot Execute Comparison on non-scalars");
+    }
+
+    if (!l.type()->Equals(r.type())) {
+      ARROW_ASSIGN_OR_RAISE(r, compute::Cast(r, l.type()));
+    }
+
+    std::vector<Datum> arguments{std::move(l), std::move(r)};
+
+    ARROW_ASSIGN_OR_RAISE(auto equal, compute::CallFunction("equal", arguments));
+
+    if (!equal.scalar()->is_valid) return NA;
+    if (equal.scalar_as<BooleanScalar>().value) return EQUAL;
+
+    ARROW_ASSIGN_OR_RAISE(auto less, compute::CallFunction("less", arguments));
+
+    if (!less.scalar()->is_valid) return NA;
+    return less.scalar_as<BooleanScalar>().value ? LESS : GREATER;
+  }
+
+  static type GetFlipped(type op) {
+    switch (op) {
+      case NA:
+        return NA;
+      case EQUAL:
+        return EQUAL;
+      case LESS:
+        return GREATER;
+      case GREATER:
+        return LESS;
+      case NOT_EQUAL:
+        return NOT_EQUAL;
+      case LESS_EQUAL:
+        return GREATER_EQUAL;
+      case GREATER_EQUAL:
+        return LESS_EQUAL;
+    }
+    DCHECK(false);
+    return NA;
+  }
+
+  static std::string GetName(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "equal";
+      case LESS:
+        return "less";
+      case GREATER:
+        return "greater";
+      case NOT_EQUAL:
+        return "not_equal";
+      case LESS_EQUAL:
+        return "less_equal";
+      case GREATER_EQUAL:
+        return "greater_equal";
+    }
+    DCHECK(false);
+    return "na";
+  }
+
+  static std::string GetOp(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "==";
+      case LESS:
+        return "<";
+      case GREATER:
+        return ">";
+      case NOT_EQUAL:
+        return "!=";
+      case LESS_EQUAL:
+        return "<=";
+      case GREATER_EQUAL:
+        return ">=";
+    }
+    DCHECK(false);
+    return "";
+  }
+};
+
+inline const compute::CastOptions* GetCastOptions(const Expression::Call& call) {
+  if (call.function_name != "cast") return nullptr;
+  return checked_cast<const compute::CastOptions*>(call.options.get());
+}
+
+inline bool IsSetLookup(const std::string& function) {
+  return function == "is_in" || function == "index_in";
+}
+
+inline bool IsSameTypesBinary(const std::string& function) {
+  if (Comparison::Get(function)) return true;
+
+  static std::unordered_set<std::string> set{"add", "subtract", "multiply", "divide"};
+
+  return set.find(function) != set.end();
+}
+
+inline const compute::SetLookupOptions* GetSetLookupOptions(
+    const Expression::Call& call) {
+  if (!IsSetLookup(call.function_name)) return nullptr;
+  return checked_cast<const compute::SetLookupOptions*>(call.options.get());
+}
+
+inline const compute::StructOptions* GetStructOptions(const Expression::Call& call) {
+  if (call.function_name != "struct") return nullptr;
+  return checked_cast<const compute::StructOptions*>(call.options.get());
+}
+
+inline const compute::StrptimeOptions* GetStrptimeOptions(const Expression::Call& call) {
+  if (call.function_name != "strptime") return nullptr;
+  return checked_cast<const compute::StrptimeOptions*>(call.options.get());
+}
+
+inline const std::shared_ptr<DataType>& GetDictionaryValueType(
+    const std::shared_ptr<DataType>& type) {
+  if (type && type->id() == Type::DICTIONARY) {
+    return checked_cast<const DictionaryType&>(*type).value_type();
+  }
+  static std::shared_ptr<DataType> null;
+  return null;
+}
+
+inline Status EnsureNotDictionary(ValueDescr* descr) {
+  if (auto value_type = GetDictionaryValueType(descr->type)) {
+    descr->type = std::move(value_type);
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Datum* datum) {
+  if (datum->type()->id() == Type::DICTIONARY) {
+    const auto& type = checked_cast<const DictionaryType&>(*datum->type()).value_type();
+    ARROW_ASSIGN_OR_RAISE(*datum, compute::Cast(*datum, type));
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Expression::Call* call) {
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto new_options = *options;
+    RETURN_NOT_OK(EnsureNotDictionary(&new_options.value_set));
+    call->options.reset(new compute::SetLookupOptions(std::move(new_options)));
+  }
+  return Status::OK();
+}
+
+inline Result<std::shared_ptr<StructScalar>> FunctionOptionsToStructScalar(
+    const Expression::Call& call) {
+  if (call.options == nullptr) {
+    return nullptr;
+  }
+
+  auto Finish = [](ScalarVector values, std::vector<std::string> names) {
+    FieldVector fields(names.size());
+    for (size_t i = 0; i < fields.size(); ++i) {
+      fields[i] = field(std::move(names[i]), values[i]->type);
+    }
+    return std::make_shared<StructScalar>(std::move(values), struct_(std::move(fields)));
+  };
+
+  if (auto options = GetSetLookupOptions(call)) {
+    if (!options->value_set.is_array()) {
+      return Status::NotImplemented("chunked value_set");
+    }
+    return Finish(
+        {
+            std::make_shared<ListScalar>(options->value_set.make_array()),
+            MakeScalar(options->skip_nulls),
+        },
+        {"value_set", "skip_nulls"});
+  }
+
+  if (call.function_name == "cast") {
+    auto options = checked_cast<const compute::CastOptions*>(call.options.get());
+    return Finish(
+        {
+            MakeNullScalar(options->to_type),
+            MakeScalar(options->allow_int_overflow),
+            MakeScalar(options->allow_time_truncate),
+            MakeScalar(options->allow_time_overflow),
+            MakeScalar(options->allow_decimal_truncate),
+            MakeScalar(options->allow_float_truncate),
+            MakeScalar(options->allow_invalid_utf8),
+        },
+        {
+            "to_type_holder",
+            "allow_int_overflow",
+            "allow_time_truncate",
+            "allow_time_overflow",
+            "allow_decimal_truncate",
+            "allow_float_truncate",
+            "allow_invalid_utf8",
+        });
+  }
+
+  return Status::NotImplemented("conversion of options for ", call.function_name);
+}
+
+inline Status FunctionOptionsFromStructScalar(const StructScalar* repr,
+                                              Expression::Call* call) {
+  if (repr == nullptr) {
+    call->options = nullptr;
+    return Status::OK();
+  }
+
+  if (IsSetLookup(call->function_name)) {
+    ARROW_ASSIGN_OR_RAISE(auto value_set, repr->field("value_set"));
+    ARROW_ASSIGN_OR_RAISE(auto skip_nulls, repr->field("skip_nulls"));
+    call->options = std::make_shared<compute::SetLookupOptions>(
+        checked_cast<const ListScalar&>(*value_set).value,
+        checked_cast<const BooleanScalar&>(*skip_nulls).value);
+    return Status::OK();
+  }
+
+  if (call->function_name == "cast") {
+    auto options = std::make_shared<compute::CastOptions>();
+    ARROW_ASSIGN_OR_RAISE(auto to_type_holder, repr->field("to_type_holder"));
+    options->to_type = to_type_holder->type;
+
+    int i = 1;
+    for (bool* opt : {
+             &options->allow_int_overflow,
+             &options->allow_time_truncate,
+             &options->allow_time_overflow,
+             &options->allow_decimal_truncate,
+             &options->allow_float_truncate,
+             &options->allow_invalid_utf8,
+         }) {
+      *opt = checked_cast<const BooleanScalar&>(*repr->value[i++]).value;
+    }
+
+    call->options = std::move(options);
+    return Status::OK();
+  }

Review comment:
       For clarity and ease of extension, perhaps this should be hiddened behind parametered helper classes, e.g.:
   ```c++
   if (IsSetLookup(call->function_name)) {
     return OptionsConverter<compute::SetLookupOptions>::ToFunctionOptions(*repr, &call->options));
   }
   if (call->function_name == "cast") {
     return OptionsConverter<compute::CastOptions>::ToFunctionOptions(*repr, &call->options));
   }
   ```
   
   And reciprocally in `FunctionOptionsToStructScalar`.

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{
+        {"equal", EQUAL},     {"not_equal", NOT_EQUAL},
+        {"less", LESS},       {"less_equal", LESS_EQUAL},
+        {"greater", GREATER}, {"greater_equal", GREATER_EQUAL},
+    };
+
+    auto it = flipped_comparisons.find(function);
+    return it != flipped_comparisons.end() ? &it->second : nullptr;
+  }
+
+  static const type* Get(const Expression& expr) {
+    if (auto call = expr.call()) {
+      return Comparison::Get(call->function_name);
+    }
+    return nullptr;
+  }
+
+  // Execute a simple Comparison between scalars, casting the RHS if types disagree
+  static Result<type> Execute(Datum l, Datum r) {
+    if (!l.is_scalar() || !r.is_scalar()) {
+      return Status::Invalid("Cannot Execute Comparison on non-scalars");
+    }
+
+    if (!l.type()->Equals(r.type())) {
+      ARROW_ASSIGN_OR_RAISE(r, compute::Cast(r, l.type()));
+    }
+
+    std::vector<Datum> arguments{std::move(l), std::move(r)};
+
+    ARROW_ASSIGN_OR_RAISE(auto equal, compute::CallFunction("equal", arguments));
+
+    if (!equal.scalar()->is_valid) return NA;
+    if (equal.scalar_as<BooleanScalar>().value) return EQUAL;
+
+    ARROW_ASSIGN_OR_RAISE(auto less, compute::CallFunction("less", arguments));
+
+    if (!less.scalar()->is_valid) return NA;
+    return less.scalar_as<BooleanScalar>().value ? LESS : GREATER;
+  }
+
+  static type GetFlipped(type op) {
+    switch (op) {
+      case NA:
+        return NA;
+      case EQUAL:
+        return EQUAL;
+      case LESS:
+        return GREATER;
+      case GREATER:
+        return LESS;
+      case NOT_EQUAL:
+        return NOT_EQUAL;
+      case LESS_EQUAL:
+        return GREATER_EQUAL;
+      case GREATER_EQUAL:
+        return LESS_EQUAL;
+    }
+    DCHECK(false);
+    return NA;
+  }
+
+  static std::string GetName(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "equal";
+      case LESS:
+        return "less";
+      case GREATER:
+        return "greater";
+      case NOT_EQUAL:
+        return "not_equal";
+      case LESS_EQUAL:
+        return "less_equal";
+      case GREATER_EQUAL:
+        return "greater_equal";
+    }
+    DCHECK(false);
+    return "na";
+  }
+
+  static std::string GetOp(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "==";
+      case LESS:
+        return "<";
+      case GREATER:
+        return ">";
+      case NOT_EQUAL:
+        return "!=";
+      case LESS_EQUAL:
+        return "<=";
+      case GREATER_EQUAL:
+        return ">=";
+    }
+    DCHECK(false);
+    return "";
+  }
+};
+
+inline const compute::CastOptions* GetCastOptions(const Expression::Call& call) {
+  if (call.function_name != "cast") return nullptr;
+  return checked_cast<const compute::CastOptions*>(call.options.get());
+}
+
+inline bool IsSetLookup(const std::string& function) {
+  return function == "is_in" || function == "index_in";
+}
+
+inline bool IsSameTypesBinary(const std::string& function) {
+  if (Comparison::Get(function)) return true;
+
+  static std::unordered_set<std::string> set{"add", "subtract", "multiply", "divide"};
+
+  return set.find(function) != set.end();
+}
+
+inline const compute::SetLookupOptions* GetSetLookupOptions(
+    const Expression::Call& call) {
+  if (!IsSetLookup(call.function_name)) return nullptr;
+  return checked_cast<const compute::SetLookupOptions*>(call.options.get());
+}
+
+inline const compute::StructOptions* GetStructOptions(const Expression::Call& call) {
+  if (call.function_name != "struct") return nullptr;
+  return checked_cast<const compute::StructOptions*>(call.options.get());
+}
+
+inline const compute::StrptimeOptions* GetStrptimeOptions(const Expression::Call& call) {
+  if (call.function_name != "strptime") return nullptr;
+  return checked_cast<const compute::StrptimeOptions*>(call.options.get());
+}
+
+inline const std::shared_ptr<DataType>& GetDictionaryValueType(
+    const std::shared_ptr<DataType>& type) {
+  if (type && type->id() == Type::DICTIONARY) {
+    return checked_cast<const DictionaryType&>(*type).value_type();
+  }
+  static std::shared_ptr<DataType> null;
+  return null;
+}
+
+inline Status EnsureNotDictionary(ValueDescr* descr) {
+  if (auto value_type = GetDictionaryValueType(descr->type)) {
+    descr->type = std::move(value_type);
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Datum* datum) {
+  if (datum->type()->id() == Type::DICTIONARY) {
+    const auto& type = checked_cast<const DictionaryType&>(*datum->type()).value_type();
+    ARROW_ASSIGN_OR_RAISE(*datum, compute::Cast(*datum, type));
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Expression::Call* call) {
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto new_options = *options;
+    RETURN_NOT_OK(EnsureNotDictionary(&new_options.value_set));
+    call->options.reset(new compute::SetLookupOptions(std::move(new_options)));
+  }
+  return Status::OK();
+}
+
+inline Result<std::shared_ptr<StructScalar>> FunctionOptionsToStructScalar(
+    const Expression::Call& call) {
+  if (call.options == nullptr) {
+    return nullptr;
+  }
+
+  auto Finish = [](ScalarVector values, std::vector<std::string> names) {

Review comment:
       This looks generally useful. Make this could be an alternate constructor or static factory of `StructScalar`?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{
+        {"equal", EQUAL},     {"not_equal", NOT_EQUAL},
+        {"less", LESS},       {"less_equal", LESS_EQUAL},
+        {"greater", GREATER}, {"greater_equal", GREATER_EQUAL},
+    };
+
+    auto it = flipped_comparisons.find(function);
+    return it != flipped_comparisons.end() ? &it->second : nullptr;
+  }
+
+  static const type* Get(const Expression& expr) {
+    if (auto call = expr.call()) {
+      return Comparison::Get(call->function_name);
+    }
+    return nullptr;
+  }
+
+  // Execute a simple Comparison between scalars, casting the RHS if types disagree
+  static Result<type> Execute(Datum l, Datum r) {
+    if (!l.is_scalar() || !r.is_scalar()) {
+      return Status::Invalid("Cannot Execute Comparison on non-scalars");
+    }
+
+    if (!l.type()->Equals(r.type())) {
+      ARROW_ASSIGN_OR_RAISE(r, compute::Cast(r, l.type()));
+    }
+
+    std::vector<Datum> arguments{std::move(l), std::move(r)};
+
+    ARROW_ASSIGN_OR_RAISE(auto equal, compute::CallFunction("equal", arguments));
+
+    if (!equal.scalar()->is_valid) return NA;
+    if (equal.scalar_as<BooleanScalar>().value) return EQUAL;
+
+    ARROW_ASSIGN_OR_RAISE(auto less, compute::CallFunction("less", arguments));
+
+    if (!less.scalar()->is_valid) return NA;
+    return less.scalar_as<BooleanScalar>().value ? LESS : GREATER;
+  }
+
+  static type GetFlipped(type op) {
+    switch (op) {
+      case NA:
+        return NA;
+      case EQUAL:
+        return EQUAL;
+      case LESS:
+        return GREATER;
+      case GREATER:
+        return LESS;
+      case NOT_EQUAL:
+        return NOT_EQUAL;
+      case LESS_EQUAL:
+        return GREATER_EQUAL;
+      case GREATER_EQUAL:
+        return LESS_EQUAL;
+    }
+    DCHECK(false);
+    return NA;
+  }
+
+  static std::string GetName(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "equal";
+      case LESS:
+        return "less";
+      case GREATER:
+        return "greater";
+      case NOT_EQUAL:
+        return "not_equal";
+      case LESS_EQUAL:
+        return "less_equal";
+      case GREATER_EQUAL:
+        return "greater_equal";
+    }
+    DCHECK(false);
+    return "na";
+  }
+
+  static std::string GetOp(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "==";
+      case LESS:
+        return "<";
+      case GREATER:
+        return ">";
+      case NOT_EQUAL:
+        return "!=";
+      case LESS_EQUAL:
+        return "<=";
+      case GREATER_EQUAL:
+        return ">=";
+    }
+    DCHECK(false);
+    return "";
+  }
+};
+
+inline const compute::CastOptions* GetCastOptions(const Expression::Call& call) {
+  if (call.function_name != "cast") return nullptr;
+  return checked_cast<const compute::CastOptions*>(call.options.get());
+}
+
+inline bool IsSetLookup(const std::string& function) {
+  return function == "is_in" || function == "index_in";
+}
+
+inline bool IsSameTypesBinary(const std::string& function) {
+  if (Comparison::Get(function)) return true;
+
+  static std::unordered_set<std::string> set{"add", "subtract", "multiply", "divide"};
+
+  return set.find(function) != set.end();
+}
+
+inline const compute::SetLookupOptions* GetSetLookupOptions(
+    const Expression::Call& call) {
+  if (!IsSetLookup(call.function_name)) return nullptr;
+  return checked_cast<const compute::SetLookupOptions*>(call.options.get());
+}
+
+inline const compute::StructOptions* GetStructOptions(const Expression::Call& call) {
+  if (call.function_name != "struct") return nullptr;
+  return checked_cast<const compute::StructOptions*>(call.options.get());
+}
+
+inline const compute::StrptimeOptions* GetStrptimeOptions(const Expression::Call& call) {
+  if (call.function_name != "strptime") return nullptr;
+  return checked_cast<const compute::StrptimeOptions*>(call.options.get());
+}
+
+inline const std::shared_ptr<DataType>& GetDictionaryValueType(
+    const std::shared_ptr<DataType>& type) {
+  if (type && type->id() == Type::DICTIONARY) {
+    return checked_cast<const DictionaryType&>(*type).value_type();
+  }
+  static std::shared_ptr<DataType> null;
+  return null;
+}
+
+inline Status EnsureNotDictionary(ValueDescr* descr) {
+  if (auto value_type = GetDictionaryValueType(descr->type)) {
+    descr->type = std::move(value_type);
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Datum* datum) {
+  if (datum->type()->id() == Type::DICTIONARY) {
+    const auto& type = checked_cast<const DictionaryType&>(*datum->type()).value_type();
+    ARROW_ASSIGN_OR_RAISE(*datum, compute::Cast(*datum, type));
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Expression::Call* call) {
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto new_options = *options;
+    RETURN_NOT_OK(EnsureNotDictionary(&new_options.value_set));
+    call->options.reset(new compute::SetLookupOptions(std::move(new_options)));
+  }
+  return Status::OK();
+}
+
+inline Result<std::shared_ptr<StructScalar>> FunctionOptionsToStructScalar(
+    const Expression::Call& call) {
+  if (call.options == nullptr) {
+    return nullptr;
+  }
+
+  auto Finish = [](ScalarVector values, std::vector<std::string> names) {
+    FieldVector fields(names.size());
+    for (size_t i = 0; i < fields.size(); ++i) {
+      fields[i] = field(std::move(names[i]), values[i]->type);
+    }
+    return std::make_shared<StructScalar>(std::move(values), struct_(std::move(fields)));
+  };
+
+  if (auto options = GetSetLookupOptions(call)) {
+    if (!options->value_set.is_array()) {
+      return Status::NotImplemented("chunked value_set");
+    }
+    return Finish(
+        {
+            std::make_shared<ListScalar>(options->value_set.make_array()),
+            MakeScalar(options->skip_nulls),
+        },
+        {"value_set", "skip_nulls"});
+  }
+
+  if (call.function_name == "cast") {
+    auto options = checked_cast<const compute::CastOptions*>(call.options.get());
+    return Finish(
+        {
+            MakeNullScalar(options->to_type),
+            MakeScalar(options->allow_int_overflow),
+            MakeScalar(options->allow_time_truncate),
+            MakeScalar(options->allow_time_overflow),
+            MakeScalar(options->allow_decimal_truncate),
+            MakeScalar(options->allow_float_truncate),
+            MakeScalar(options->allow_invalid_utf8),
+        },
+        {
+            "to_type_holder",
+            "allow_int_overflow",
+            "allow_time_truncate",
+            "allow_time_overflow",
+            "allow_decimal_truncate",
+            "allow_float_truncate",
+            "allow_invalid_utf8",
+        });
+  }
+
+  return Status::NotImplemented("conversion of options for ", call.function_name);
+}
+
+inline Status FunctionOptionsFromStructScalar(const StructScalar* repr,
+                                              Expression::Call* call) {
+  if (repr == nullptr) {
+    call->options = nullptr;
+    return Status::OK();
+  }
+
+  if (IsSetLookup(call->function_name)) {
+    ARROW_ASSIGN_OR_RAISE(auto value_set, repr->field("value_set"));
+    ARROW_ASSIGN_OR_RAISE(auto skip_nulls, repr->field("skip_nulls"));
+    call->options = std::make_shared<compute::SetLookupOptions>(
+        checked_cast<const ListScalar&>(*value_set).value,
+        checked_cast<const BooleanScalar&>(*skip_nulls).value);
+    return Status::OK();
+  }
+
+  if (call->function_name == "cast") {
+    auto options = std::make_shared<compute::CastOptions>();
+    ARROW_ASSIGN_OR_RAISE(auto to_type_holder, repr->field("to_type_holder"));
+    options->to_type = to_type_holder->type;
+
+    int i = 1;
+    for (bool* opt : {
+             &options->allow_int_overflow,
+             &options->allow_time_truncate,
+             &options->allow_time_overflow,
+             &options->allow_decimal_truncate,
+             &options->allow_float_truncate,
+             &options->allow_invalid_utf8,
+         }) {
+      *opt = checked_cast<const BooleanScalar&>(*repr->value[i++]).value;
+    }
+
+    call->options = std::move(options);
+    return Status::OK();
+  }
+
+  return Status::NotImplemented("conversion of options for ", call->function_name);
+}
+
+struct FlattenedAssociativeChain {
+  bool was_left_folded = true;
+  std::vector<Expression> exprs, fringe;
+
+  explicit FlattenedAssociativeChain(Expression expr) : exprs{std::move(expr)} {

Review comment:
       What does this do? Add a comment?

##########
File path: cpp/src/arrow/dataset/expression_internal.h
##########
@@ -0,0 +1,465 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+#include <vector>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/registry.h"
+#include "arrow/record_batch.h"
+#include "arrow/table.h"
+#include "arrow/util/logging.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace dataset {
+
+bool Identical(const Expression& l, const Expression& r) { return l.impl_ == r.impl_; }
+
+const Expression::Call* CallNotNull(const Expression& expr) {
+  auto call = expr.call();
+  DCHECK_NE(call, nullptr);
+  return call;
+}
+
+inline void GetAllFieldRefs(const Expression& expr,
+                            std::unordered_set<FieldRef, FieldRef::Hash>* refs) {
+  if (auto lit = expr.literal()) return;
+
+  if (auto ref = expr.field_ref()) {
+    refs->emplace(*ref);
+    return;
+  }
+
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    GetAllFieldRefs(arg, refs);
+  }
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Expression>& exprs) {
+  std::vector<ValueDescr> descrs(exprs.size());
+  for (size_t i = 0; i < exprs.size(); ++i) {
+    DCHECK(exprs[i].IsBound());
+    descrs[i] = exprs[i].descr();
+  }
+  return descrs;
+}
+
+inline std::vector<ValueDescr> GetDescriptors(const std::vector<Datum>& values) {
+  std::vector<ValueDescr> descrs(values.size());
+  for (size_t i = 0; i < values.size(); ++i) {
+    descrs[i] = values[i].descr();
+  }
+  return descrs;
+}
+
+struct FieldPathGetDatumImpl {
+  template <typename T, typename = decltype(FieldPath{}.Get(std::declval<const T&>()))>
+  Result<Datum> operator()(const std::shared_ptr<T>& ptr) {
+    return path_.Get(*ptr).template As<Datum>();
+  }
+
+  template <typename T>
+  Result<Datum> operator()(const T&) {
+    return Status::NotImplemented("FieldPath::Get() into Datum ", datum_.ToString());
+  }
+
+  const Datum& datum_;
+  const FieldPath& path_;
+};
+
+inline Result<Datum> GetDatumField(const FieldRef& ref, const Datum& input) {
+  Datum field;
+
+  FieldPath path;
+  if (auto type = input.type()) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.type()));
+  } else if (input.kind() == Datum::RECORD_BATCH) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.record_batch()->schema()));
+  } else if (input.kind() == Datum::TABLE) {
+    ARROW_ASSIGN_OR_RAISE(path, ref.FindOneOrNone(*input.table()->schema()));
+  }
+
+  if (path) {
+    ARROW_ASSIGN_OR_RAISE(field,
+                          util::visit(FieldPathGetDatumImpl{input, path}, input.value));
+  }
+
+  if (field == Datum{}) {
+    field = Datum(std::make_shared<NullScalar>());
+  }
+
+  return field;
+}
+
+struct Comparison {
+  enum type {
+    NA = 0,
+    EQUAL = 1,
+    LESS = 2,
+    GREATER = 4,
+    NOT_EQUAL = LESS | GREATER,
+    LESS_EQUAL = LESS | EQUAL,
+    GREATER_EQUAL = GREATER | EQUAL,
+  };
+
+  static const type* Get(const std::string& function) {
+    static std::unordered_map<std::string, type> flipped_comparisons{
+        {"equal", EQUAL},     {"not_equal", NOT_EQUAL},
+        {"less", LESS},       {"less_equal", LESS_EQUAL},
+        {"greater", GREATER}, {"greater_equal", GREATER_EQUAL},
+    };
+
+    auto it = flipped_comparisons.find(function);
+    return it != flipped_comparisons.end() ? &it->second : nullptr;
+  }
+
+  static const type* Get(const Expression& expr) {
+    if (auto call = expr.call()) {
+      return Comparison::Get(call->function_name);
+    }
+    return nullptr;
+  }
+
+  // Execute a simple Comparison between scalars, casting the RHS if types disagree
+  static Result<type> Execute(Datum l, Datum r) {
+    if (!l.is_scalar() || !r.is_scalar()) {
+      return Status::Invalid("Cannot Execute Comparison on non-scalars");
+    }
+
+    if (!l.type()->Equals(r.type())) {
+      ARROW_ASSIGN_OR_RAISE(r, compute::Cast(r, l.type()));
+    }
+
+    std::vector<Datum> arguments{std::move(l), std::move(r)};
+
+    ARROW_ASSIGN_OR_RAISE(auto equal, compute::CallFunction("equal", arguments));
+
+    if (!equal.scalar()->is_valid) return NA;
+    if (equal.scalar_as<BooleanScalar>().value) return EQUAL;
+
+    ARROW_ASSIGN_OR_RAISE(auto less, compute::CallFunction("less", arguments));
+
+    if (!less.scalar()->is_valid) return NA;
+    return less.scalar_as<BooleanScalar>().value ? LESS : GREATER;
+  }
+
+  static type GetFlipped(type op) {
+    switch (op) {
+      case NA:
+        return NA;
+      case EQUAL:
+        return EQUAL;
+      case LESS:
+        return GREATER;
+      case GREATER:
+        return LESS;
+      case NOT_EQUAL:
+        return NOT_EQUAL;
+      case LESS_EQUAL:
+        return GREATER_EQUAL;
+      case GREATER_EQUAL:
+        return LESS_EQUAL;
+    }
+    DCHECK(false);
+    return NA;
+  }
+
+  static std::string GetName(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "equal";
+      case LESS:
+        return "less";
+      case GREATER:
+        return "greater";
+      case NOT_EQUAL:
+        return "not_equal";
+      case LESS_EQUAL:
+        return "less_equal";
+      case GREATER_EQUAL:
+        return "greater_equal";
+    }
+    DCHECK(false);
+    return "na";
+  }
+
+  static std::string GetOp(type op) {
+    switch (op) {
+      case NA:
+        DCHECK(false) << "unreachable";
+        break;
+      case EQUAL:
+        return "==";
+      case LESS:
+        return "<";
+      case GREATER:
+        return ">";
+      case NOT_EQUAL:
+        return "!=";
+      case LESS_EQUAL:
+        return "<=";
+      case GREATER_EQUAL:
+        return ">=";
+    }
+    DCHECK(false);
+    return "";
+  }
+};
+
+inline const compute::CastOptions* GetCastOptions(const Expression::Call& call) {
+  if (call.function_name != "cast") return nullptr;
+  return checked_cast<const compute::CastOptions*>(call.options.get());
+}
+
+inline bool IsSetLookup(const std::string& function) {
+  return function == "is_in" || function == "index_in";
+}
+
+inline bool IsSameTypesBinary(const std::string& function) {
+  if (Comparison::Get(function)) return true;
+
+  static std::unordered_set<std::string> set{"add", "subtract", "multiply", "divide"};
+
+  return set.find(function) != set.end();
+}
+
+inline const compute::SetLookupOptions* GetSetLookupOptions(
+    const Expression::Call& call) {
+  if (!IsSetLookup(call.function_name)) return nullptr;
+  return checked_cast<const compute::SetLookupOptions*>(call.options.get());
+}
+
+inline const compute::StructOptions* GetStructOptions(const Expression::Call& call) {
+  if (call.function_name != "struct") return nullptr;
+  return checked_cast<const compute::StructOptions*>(call.options.get());
+}
+
+inline const compute::StrptimeOptions* GetStrptimeOptions(const Expression::Call& call) {
+  if (call.function_name != "strptime") return nullptr;
+  return checked_cast<const compute::StrptimeOptions*>(call.options.get());
+}
+
+inline const std::shared_ptr<DataType>& GetDictionaryValueType(
+    const std::shared_ptr<DataType>& type) {
+  if (type && type->id() == Type::DICTIONARY) {
+    return checked_cast<const DictionaryType&>(*type).value_type();
+  }
+  static std::shared_ptr<DataType> null;
+  return null;
+}
+
+inline Status EnsureNotDictionary(ValueDescr* descr) {
+  if (auto value_type = GetDictionaryValueType(descr->type)) {
+    descr->type = std::move(value_type);
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Datum* datum) {
+  if (datum->type()->id() == Type::DICTIONARY) {
+    const auto& type = checked_cast<const DictionaryType&>(*datum->type()).value_type();
+    ARROW_ASSIGN_OR_RAISE(*datum, compute::Cast(*datum, type));
+  }
+  return Status::OK();
+}
+
+inline Status EnsureNotDictionary(Expression::Call* call) {
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto new_options = *options;
+    RETURN_NOT_OK(EnsureNotDictionary(&new_options.value_set));
+    call->options.reset(new compute::SetLookupOptions(std::move(new_options)));
+  }
+  return Status::OK();
+}
+
+inline Result<std::shared_ptr<StructScalar>> FunctionOptionsToStructScalar(
+    const Expression::Call& call) {
+  if (call.options == nullptr) {
+    return nullptr;
+  }
+
+  auto Finish = [](ScalarVector values, std::vector<std::string> names) {
+    FieldVector fields(names.size());
+    for (size_t i = 0; i < fields.size(); ++i) {
+      fields[i] = field(std::move(names[i]), values[i]->type);
+    }
+    return std::make_shared<StructScalar>(std::move(values), struct_(std::move(fields)));
+  };
+
+  if (auto options = GetSetLookupOptions(call)) {
+    if (!options->value_set.is_array()) {
+      return Status::NotImplemented("chunked value_set");
+    }
+    return Finish(
+        {
+            std::make_shared<ListScalar>(options->value_set.make_array()),
+            MakeScalar(options->skip_nulls),
+        },
+        {"value_set", "skip_nulls"});
+  }
+
+  if (call.function_name == "cast") {
+    auto options = checked_cast<const compute::CastOptions*>(call.options.get());
+    return Finish(
+        {
+            MakeNullScalar(options->to_type),
+            MakeScalar(options->allow_int_overflow),
+            MakeScalar(options->allow_time_truncate),
+            MakeScalar(options->allow_time_overflow),
+            MakeScalar(options->allow_decimal_truncate),
+            MakeScalar(options->allow_float_truncate),
+            MakeScalar(options->allow_invalid_utf8),
+        },
+        {
+            "to_type_holder",
+            "allow_int_overflow",
+            "allow_time_truncate",
+            "allow_time_overflow",
+            "allow_decimal_truncate",
+            "allow_float_truncate",
+            "allow_invalid_utf8",
+        });
+  }
+
+  return Status::NotImplemented("conversion of options for ", call.function_name);
+}
+
+inline Status FunctionOptionsFromStructScalar(const StructScalar* repr,
+                                              Expression::Call* call) {
+  if (repr == nullptr) {
+    call->options = nullptr;
+    return Status::OK();
+  }
+
+  if (IsSetLookup(call->function_name)) {
+    ARROW_ASSIGN_OR_RAISE(auto value_set, repr->field("value_set"));
+    ARROW_ASSIGN_OR_RAISE(auto skip_nulls, repr->field("skip_nulls"));
+    call->options = std::make_shared<compute::SetLookupOptions>(
+        checked_cast<const ListScalar&>(*value_set).value,
+        checked_cast<const BooleanScalar&>(*skip_nulls).value);
+    return Status::OK();
+  }
+
+  if (call->function_name == "cast") {
+    auto options = std::make_shared<compute::CastOptions>();
+    ARROW_ASSIGN_OR_RAISE(auto to_type_holder, repr->field("to_type_holder"));
+    options->to_type = to_type_holder->type;
+
+    int i = 1;
+    for (bool* opt : {
+             &options->allow_int_overflow,
+             &options->allow_time_truncate,
+             &options->allow_time_overflow,
+             &options->allow_decimal_truncate,
+             &options->allow_float_truncate,
+             &options->allow_invalid_utf8,
+         }) {
+      *opt = checked_cast<const BooleanScalar&>(*repr->value[i++]).value;
+    }
+
+    call->options = std::move(options);
+    return Status::OK();
+  }
+
+  return Status::NotImplemented("conversion of options for ", call->function_name);
+}
+
+struct FlattenedAssociativeChain {
+  bool was_left_folded = true;
+  std::vector<Expression> exprs, fringe;
+
+  explicit FlattenedAssociativeChain(Expression expr) : exprs{std::move(expr)} {
+    auto call = CallNotNull(exprs.back());
+    fringe = call->arguments;
+
+    auto it = fringe.begin();
+
+    while (it != fringe.end()) {
+      auto sub_call = it->call();
+      if (!sub_call || sub_call->function_name != call->function_name) {
+        ++it;
+        continue;
+      }
+
+      if (it != fringe.begin()) {
+        was_left_folded = false;
+      }
+
+      exprs.push_back(std::move(*it));
+      it = fringe.erase(it);
+      it = fringe.insert(it, sub_call->arguments.begin(), sub_call->arguments.end());
+      // NB: no increment so we hit sub_call's first argument next iteration
+    }
+
+    DCHECK(std::all_of(exprs.begin(), exprs.end(), [](const Expression& expr) {
+      return CallNotNull(expr)->options == nullptr;
+    }));
+  }
+};
+
+inline Result<std::shared_ptr<compute::Function>> GetFunction(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (call.function_name != "cast") {
+    return exec_context->func_registry()->GetFunction(call.function_name);
+  }
+  // XXX this special case is strange; why not make "cast" a ScalarFunction?

Review comment:
       Because its output type depends on the options, not only the input types, I presume?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced

Review comment:
       Why does it need fixing?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);

Review comment:
       Ideally we would have `FunctionOptions::Equals`...

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally

Review comment:
       Well, the problem is that you may be casting a large type to a narrower type. We would need some type unification logic.

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));

Review comment:
       Hmm... why does it change anything to "bind" with an empty descr?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));

Review comment:
       `InitKernelState` recreates a new kernel context. Should it keep the same one?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.

Review comment:
       Perhaps it should? It would also be far faster in practice (only do the set lookup on the dictionary and take from the results...).

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {

Review comment:
       This calls `IsScalarExpression` at each level of the expression tree, and `IsScalarExpression` itself is recursive, so will result in O(N^2) expression node visits.

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.

Review comment:
       But why would anyone pass a dictionary as a value set? Especially as a value set, by definition, doesn't care about duplicates.

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());

Review comment:
       Hmm, so referring a non-existing field is ok? That seems a bit fragile in the presence of e.g. typos.

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));

Review comment:
       Is this required for any actual use case?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();
+
+  compute::KernelContext kernel_context(exec_context);
+  kernel_context.SetState(call->kernel_state.get());
+
+  auto kernel = call->kernel;
+  auto descrs = GetDescriptors(arguments);
+  auto options = call->options.get();
+  RETURN_NOT_OK(executor->Init(&kernel_context, {kernel, descrs, options}));
+
+  auto listener = std::make_shared<compute::detail::DatumAccumulator>();

Review comment:
       Hopefully we can have a compute API that avoids going through this...

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();

Review comment:
       We really need a compute API that allow to executor a `Kernel` over a vector of `Datum` inputs together with some `FunctionOptions`. Perhaps open a JIRA for that?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {

Review comment:
       It seems that we may want a `virtual Result<std::unique_ptr<FunctionOptions>> FunctionOptions::CastFor(const std::shared_ptr<DataType>& input_type)`?
   
   (returning nullptr by default, meaning unchanged)

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,

Review comment:
       At some point, put all private stuff in the anonymous namespace.

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {

Review comment:
       Unless expressions are only meant to support a small frozen subset of compute functions?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();
+
+  compute::KernelContext kernel_context(exec_context);
+  kernel_context.SetState(call->kernel_state.get());
+
+  auto kernel = call->kernel;
+  auto descrs = GetDescriptors(arguments);
+  auto options = call->options.get();
+  RETURN_NOT_OK(executor->Init(&kernel_context, {kernel, descrs, options}));
+
+  auto listener = std::make_shared<compute::detail::DatumAccumulator>();
+  RETURN_NOT_OK(executor->Execute(arguments, listener.get()));
+  return executor->WrapResults(arguments, listener->values());
+}
+
+std::array<std::pair<const Expression&, const Expression&>, 2>
+ArgumentsAndFlippedArguments(const Expression::Call& call) {
+  DCHECK_EQ(call.arguments.size(), 2);
+  return {std::pair<const Expression&, const Expression&>{call.arguments[0],
+                                                          call.arguments[1]},
+          std::pair<const Expression&, const Expression&>{call.arguments[1],
+                                                          call.arguments[0]}};
+}
+
+template <typename BinOp, typename It,
+          typename Out = typename std::iterator_traits<It>::value_type>
+util::optional<Out> FoldLeft(It begin, It end, const BinOp& bin_op) {
+  if (begin == end) return util::nullopt;
+
+  Out folded = std::move(*begin++);
+  while (begin != end) {
+    folded = bin_op(std::move(folded), std::move(*begin++));
+  }
+  return folded;
+}
+
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  if (call.function && call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
+  }
+  return util::nullopt;
+}
+
+bool DefinitelyNotNull(const Expression& expr) {
+  DCHECK(expr.IsBound());
+
+  if (expr.literal()) {
+    return !expr.IsNullLiteral();
+  }
+
+  if (expr.field_ref()) return false;
+
+  auto call = CallNotNull(expr);
+  if (auto null_handling = GetNullHandling(*call)) {
+    if (null_handling == compute::NullHandling::OUTPUT_NOT_NULL) {
+      return true;
+    }
+    if (null_handling == compute::NullHandling::INTERSECTION) {
+      return std::all_of(call->arguments.begin(), call->arguments.end(),
+                         DefinitelyNotNull);
+    }
+  }
+
+  return false;
+}
+
+std::vector<FieldRef> FieldsInExpression(const Expression& expr) {
+  if (auto lit = expr.literal()) return {};
+
+  if (auto ref = expr.field_ref()) {
+    return {*ref};
+  }
+
+  std::vector<FieldRef> fields;
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    auto argument_fields = FieldsInExpression(arg);
+    std::move(argument_fields.begin(), argument_fields.end(), std::back_inserter(fields));
+  }
+  return fields;
+}
+
+Result<Expression> FoldConstants(Expression expr) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [](Expression expr, ...) -> Result<Expression> {
+        auto call = CallNotNull(expr);
+        if (std::all_of(call->arguments.begin(), call->arguments.end(),
+                        [](const Expression& argument) { return argument.literal(); })) {
+          // all arguments are literal; we can evaluate this subexpression *now*
+          static const Datum ignored_input;
+          ARROW_ASSIGN_OR_RAISE(Datum constant,
+                                ExecuteScalarExpression(expr, ignored_input));
+
+          return literal(std::move(constant));
+        }
+
+        // XXX the following should probably be in a registry of passes instead
+        // of inline
+
+        if (GetNullHandling(*call) == compute::NullHandling::INTERSECTION) {
+          // kernels which always produce intersected validity can be resolved
+          // to null *now* if any of their inputs is a null literal
+          for (const auto& argument : call->arguments) {
+            if (argument.IsNullLiteral()) {
+              return argument;
+            }
+          }
+        }
+
+        if (call->function_name == "and_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // true and x == x
+            if (args.first == literal(true)) return args.second;
+
+            // false and x == false
+            if (args.first == literal(false)) return args.first;
+
+            // x and x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        if (call->function_name == "or_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // false or x == x
+            if (args.first == literal(false)) return args.second;
+
+            // true or x == true
+            if (args.first == literal(true)) return args.first;
+
+            // x or x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        return expr;
+      });
+}
+
+inline std::vector<Expression> GuaranteeConjunctionMembers(
+    const Expression& guaranteed_true_predicate) {
+  auto guarantee = guaranteed_true_predicate.call();
+  if (!guarantee || guarantee->function_name != "and_kleene") {
+    return {guaranteed_true_predicate};
+  }
+  return FlattenedAssociativeChain(guaranteed_true_predicate).fringe;
+}
+
+// Conjunction members which are represented in known_values are erased from
+// conjunction_members
+Status ExtractKnownFieldValuesImpl(
+    std::vector<Expression>* conjunction_members,
+    std::unordered_map<FieldRef, Datum, FieldRef::Hash>* known_values) {
+  auto unconsumed_end =
+      std::partition(conjunction_members->begin(), conjunction_members->end(),
+                     [](const Expression& expr) {
+                       // search for an equality conditions between a field and a literal
+                       auto call = expr.call();
+                       if (!call) return true;
+
+                       if (call->function_name == "equal") {
+                         auto ref = call->arguments[0].field_ref();
+                         auto lit = call->arguments[1].literal();
+                         return !(ref && lit);
+                       }
+
+                       return true;
+                     });
+
+  for (auto it = unconsumed_end; it != conjunction_members->end(); ++it) {
+    auto call = CallNotNull(*it);
+
+    auto ref = call->arguments[0].field_ref();
+    auto lit = call->arguments[1].literal();
+
+    auto it_success = known_values->emplace(*ref, *lit);
+    if (it_success.second) continue;
+
+    // A value was already known for ref; check it
+    auto ref_lit = it_success.first;
+    if (*lit != ref_lit->second) {
+      return Status::Invalid("Conflicting guarantees: (", ref->ToString(),
+                             " == ", lit->ToString(), ") vs (", ref->ToString(),
+                             " == ", ref_lit->second.ToString());
+    }
+  }
+
+  conjunction_members->erase(unconsumed_end, conjunction_members->end());
+
+  return Status::OK();
+}
+
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(
+    const Expression& guaranteed_true_predicate) {
+  auto conjunction_members = GuaranteeConjunctionMembers(guaranteed_true_predicate);
+  std::unordered_map<FieldRef, Datum, FieldRef::Hash> known_values;
+  RETURN_NOT_OK(ExtractKnownFieldValuesImpl(&conjunction_members, &known_values));
+  return known_values;
+}
+
+Result<Expression> ReplaceFieldsWithKnownValues(
+    const std::unordered_map<FieldRef, Datum, FieldRef::Hash>& known_values,
+    Expression expr) {
+  if (!expr.IsBound()) {
+    return Status::Invalid(
+        "ReplaceFieldsWithKnownValues called on an unbound Expression");
+  }
+
+  return Modify(
+      std::move(expr),
+      [&known_values](Expression expr) -> Result<Expression> {
+        if (auto ref = expr.field_ref()) {
+          auto it = known_values.find(*ref);
+          if (it != known_values.end()) {
+            ARROW_ASSIGN_OR_RAISE(Datum lit,
+                                  compute::Cast(it->second, expr.descr().type));
+            return literal(std::move(lit));
+          }
+        }
+        return expr;
+      },
+      [](Expression expr, ...) { return expr; });
+}
+
+inline bool IsBinaryAssociativeCommutative(const Expression::Call& call) {
+  static std::unordered_set<std::string> binary_associative_commutative{
+      "and",      "or",  "and_kleene",       "or_kleene",  "xor",
+      "multiply", "add", "multiply_checked", "add_checked"};
+
+  auto it = binary_associative_commutative.find(call.function_name);
+  return it != binary_associative_commutative.end();
+}
+
+Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Canonicalize(std::move(expr), &exec_context);
+  }
+
+  // If potentially reconstructing more deeply than a call's immediate arguments
+  // (for example, when reorganizing an associative chain), add expressions to this set to
+  // avoid unnecessary work
+  struct {
+    std::unordered_set<Expression, Expression::Hash> set_;
+
+    bool operator()(const Expression& expr) const {
+      return set_.find(expr) != set_.end();
+    }
+
+    void Add(std::vector<Expression> exprs) {
+      std::move(exprs.begin(), exprs.end(), std::inserter(set_, set_.end()));
+    }
+  } AlreadyCanonicalized;
+
+  return Modify(
+      std::move(expr),
+      [&AlreadyCanonicalized, exec_context](Expression expr) -> Result<Expression> {
+        auto call = expr.call();
+        if (!call) return expr;
+
+        if (AlreadyCanonicalized(expr)) return expr;

Review comment:
       Note that Expression hash function is O(nodes) and you're calling it at each level... I think memoizing the hash value inside the Expression may be a good idea.

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();
+
+  compute::KernelContext kernel_context(exec_context);
+  kernel_context.SetState(call->kernel_state.get());
+
+  auto kernel = call->kernel;
+  auto descrs = GetDescriptors(arguments);
+  auto options = call->options.get();
+  RETURN_NOT_OK(executor->Init(&kernel_context, {kernel, descrs, options}));
+
+  auto listener = std::make_shared<compute::detail::DatumAccumulator>();
+  RETURN_NOT_OK(executor->Execute(arguments, listener.get()));
+  return executor->WrapResults(arguments, listener->values());
+}
+
+std::array<std::pair<const Expression&, const Expression&>, 2>
+ArgumentsAndFlippedArguments(const Expression::Call& call) {
+  DCHECK_EQ(call.arguments.size(), 2);
+  return {std::pair<const Expression&, const Expression&>{call.arguments[0],
+                                                          call.arguments[1]},
+          std::pair<const Expression&, const Expression&>{call.arguments[1],
+                                                          call.arguments[0]}};
+}
+
+template <typename BinOp, typename It,
+          typename Out = typename std::iterator_traits<It>::value_type>
+util::optional<Out> FoldLeft(It begin, It end, const BinOp& bin_op) {
+  if (begin == end) return util::nullopt;
+
+  Out folded = std::move(*begin++);
+  while (begin != end) {
+    folded = bin_op(std::move(folded), std::move(*begin++));
+  }
+  return folded;
+}
+
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  if (call.function && call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
+  }
+  return util::nullopt;
+}
+
+bool DefinitelyNotNull(const Expression& expr) {
+  DCHECK(expr.IsBound());
+
+  if (expr.literal()) {
+    return !expr.IsNullLiteral();
+  }
+
+  if (expr.field_ref()) return false;
+
+  auto call = CallNotNull(expr);
+  if (auto null_handling = GetNullHandling(*call)) {
+    if (null_handling == compute::NullHandling::OUTPUT_NOT_NULL) {
+      return true;
+    }
+    if (null_handling == compute::NullHandling::INTERSECTION) {
+      return std::all_of(call->arguments.begin(), call->arguments.end(),
+                         DefinitelyNotNull);
+    }
+  }
+
+  return false;
+}
+
+std::vector<FieldRef> FieldsInExpression(const Expression& expr) {
+  if (auto lit = expr.literal()) return {};
+
+  if (auto ref = expr.field_ref()) {
+    return {*ref};
+  }
+
+  std::vector<FieldRef> fields;
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    auto argument_fields = FieldsInExpression(arg);
+    std::move(argument_fields.begin(), argument_fields.end(), std::back_inserter(fields));
+  }
+  return fields;
+}
+
+Result<Expression> FoldConstants(Expression expr) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [](Expression expr, ...) -> Result<Expression> {
+        auto call = CallNotNull(expr);
+        if (std::all_of(call->arguments.begin(), call->arguments.end(),
+                        [](const Expression& argument) { return argument.literal(); })) {
+          // all arguments are literal; we can evaluate this subexpression *now*
+          static const Datum ignored_input;
+          ARROW_ASSIGN_OR_RAISE(Datum constant,
+                                ExecuteScalarExpression(expr, ignored_input));
+
+          return literal(std::move(constant));
+        }
+
+        // XXX the following should probably be in a registry of passes instead
+        // of inline
+
+        if (GetNullHandling(*call) == compute::NullHandling::INTERSECTION) {
+          // kernels which always produce intersected validity can be resolved
+          // to null *now* if any of their inputs is a null literal
+          for (const auto& argument : call->arguments) {
+            if (argument.IsNullLiteral()) {
+              return argument;
+            }
+          }
+        }
+
+        if (call->function_name == "and_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // true and x == x
+            if (args.first == literal(true)) return args.second;
+
+            // false and x == false
+            if (args.first == literal(false)) return args.first;
+
+            // x and x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        if (call->function_name == "or_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // false or x == x
+            if (args.first == literal(false)) return args.second;
+
+            // true or x == true
+            if (args.first == literal(true)) return args.first;
+
+            // x or x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        return expr;
+      });
+}
+
+inline std::vector<Expression> GuaranteeConjunctionMembers(
+    const Expression& guaranteed_true_predicate) {
+  auto guarantee = guaranteed_true_predicate.call();
+  if (!guarantee || guarantee->function_name != "and_kleene") {
+    return {guaranteed_true_predicate};
+  }
+  return FlattenedAssociativeChain(guaranteed_true_predicate).fringe;
+}
+
+// Conjunction members which are represented in known_values are erased from
+// conjunction_members
+Status ExtractKnownFieldValuesImpl(
+    std::vector<Expression>* conjunction_members,
+    std::unordered_map<FieldRef, Datum, FieldRef::Hash>* known_values) {
+  auto unconsumed_end =
+      std::partition(conjunction_members->begin(), conjunction_members->end(),
+                     [](const Expression& expr) {
+                       // search for an equality conditions between a field and a literal
+                       auto call = expr.call();
+                       if (!call) return true;
+
+                       if (call->function_name == "equal") {
+                         auto ref = call->arguments[0].field_ref();
+                         auto lit = call->arguments[1].literal();
+                         return !(ref && lit);
+                       }
+
+                       return true;
+                     });
+
+  for (auto it = unconsumed_end; it != conjunction_members->end(); ++it) {
+    auto call = CallNotNull(*it);
+
+    auto ref = call->arguments[0].field_ref();
+    auto lit = call->arguments[1].literal();
+
+    auto it_success = known_values->emplace(*ref, *lit);
+    if (it_success.second) continue;
+
+    // A value was already known for ref; check it
+    auto ref_lit = it_success.first;
+    if (*lit != ref_lit->second) {
+      return Status::Invalid("Conflicting guarantees: (", ref->ToString(),
+                             " == ", lit->ToString(), ") vs (", ref->ToString(),
+                             " == ", ref_lit->second.ToString());
+    }
+  }
+
+  conjunction_members->erase(unconsumed_end, conjunction_members->end());
+
+  return Status::OK();
+}
+
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(
+    const Expression& guaranteed_true_predicate) {
+  auto conjunction_members = GuaranteeConjunctionMembers(guaranteed_true_predicate);
+  std::unordered_map<FieldRef, Datum, FieldRef::Hash> known_values;
+  RETURN_NOT_OK(ExtractKnownFieldValuesImpl(&conjunction_members, &known_values));
+  return known_values;
+}
+
+Result<Expression> ReplaceFieldsWithKnownValues(
+    const std::unordered_map<FieldRef, Datum, FieldRef::Hash>& known_values,
+    Expression expr) {
+  if (!expr.IsBound()) {
+    return Status::Invalid(
+        "ReplaceFieldsWithKnownValues called on an unbound Expression");
+  }
+
+  return Modify(
+      std::move(expr),
+      [&known_values](Expression expr) -> Result<Expression> {
+        if (auto ref = expr.field_ref()) {
+          auto it = known_values.find(*ref);
+          if (it != known_values.end()) {
+            ARROW_ASSIGN_OR_RAISE(Datum lit,
+                                  compute::Cast(it->second, expr.descr().type));
+            return literal(std::move(lit));
+          }
+        }
+        return expr;
+      },
+      [](Expression expr, ...) { return expr; });
+}
+
+inline bool IsBinaryAssociativeCommutative(const Expression::Call& call) {
+  static std::unordered_set<std::string> binary_associative_commutative{
+      "and",      "or",  "and_kleene",       "or_kleene",  "xor",
+      "multiply", "add", "multiply_checked", "add_checked"};
+
+  auto it = binary_associative_commutative.find(call.function_name);
+  return it != binary_associative_commutative.end();
+}
+
+Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Canonicalize(std::move(expr), &exec_context);
+  }
+
+  // If potentially reconstructing more deeply than a call's immediate arguments
+  // (for example, when reorganizing an associative chain), add expressions to this set to
+  // avoid unnecessary work
+  struct {
+    std::unordered_set<Expression, Expression::Hash> set_;
+
+    bool operator()(const Expression& expr) const {
+      return set_.find(expr) != set_.end();
+    }
+
+    void Add(std::vector<Expression> exprs) {
+      std::move(exprs.begin(), exprs.end(), std::inserter(set_, set_.end()));
+    }
+  } AlreadyCanonicalized;
+
+  return Modify(
+      std::move(expr),
+      [&AlreadyCanonicalized, exec_context](Expression expr) -> Result<Expression> {
+        auto call = expr.call();
+        if (!call) return expr;
+
+        if (AlreadyCanonicalized(expr)) return expr;
+
+        if (IsBinaryAssociativeCommutative(*call)) {
+          struct {
+            int Priority(const Expression& operand) const {
+              // order literals first, starting with nulls
+              if (operand.IsNullLiteral()) return 0;
+              if (operand.literal()) return 1;
+              return 2;
+            }
+            bool operator()(const Expression& l, const Expression& r) const {
+              return Priority(l) < Priority(r);
+            }
+          } CanonicalOrdering;
+
+          FlattenedAssociativeChain chain(expr);
+          if (chain.was_left_folded &&
+              std::is_sorted(chain.fringe.begin(), chain.fringe.end(),
+                             CanonicalOrdering)) {
+            AlreadyCanonicalized.Add(std::move(chain.exprs));
+            return expr;
+          }
+
+          std::stable_sort(chain.fringe.begin(), chain.fringe.end(), CanonicalOrdering);
+
+          // fold the chain back up
+          auto folded =
+              FoldLeft(chain.fringe.begin(), chain.fringe.end(),
+                       [call, &AlreadyCanonicalized](Expression l, Expression r) {
+                         auto canonicalized_call = *call;
+                         canonicalized_call.arguments = {std::move(l), std::move(r)};
+                         Expression expr(std::move(canonicalized_call));
+                         AlreadyCanonicalized.Add({expr});
+                         return expr;
+                       });
+          return std::move(*folded);
+        }
+
+        if (auto cmp = Comparison::Get(call->function_name)) {
+          if (call->arguments[0].literal() && !call->arguments[1].literal()) {
+            // ensure that literals are on comparisons' RHS
+            auto flipped_call = *call;
+            flipped_call.function_name =
+                Comparison::GetName(Comparison::GetFlipped(*cmp));
+            // look up the flipped kernel
+            // TODO extract a helper for use here and in Bind
+            ARROW_ASSIGN_OR_RAISE(
+                auto function,
+                exec_context->func_registry()->GetFunction(flipped_call.function_name));
+
+            auto descrs = GetDescriptors(flipped_call.arguments);
+            ARROW_ASSIGN_OR_RAISE(flipped_call.kernel, function->DispatchExact(descrs));
+
+            std::swap(flipped_call.arguments[0], flipped_call.arguments[1]);
+            return Expression(std::move(flipped_call));
+          }
+        }
+
+        return expr;
+      },
+      [](Expression expr, ...) { return expr; });
+}
+
+Result<Expression> DirectComparisonSimplification(Expression expr,
+                                                  const Expression::Call& guarantee) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [&guarantee](Expression expr, ...) -> Result<Expression> {
+        auto call = expr.call();
+        if (!call) return expr;
+
+        // Ensure both calls are comparisons with equal LHS and scalar RHS
+        auto cmp = Comparison::Get(expr);
+        auto cmp_guarantee = Comparison::Get(guarantee.function_name);
+        if (!cmp || !cmp_guarantee) return expr;
+
+        if (call->arguments[0] != guarantee.arguments[0]) return expr;
+
+        auto rhs = call->arguments[1].literal();
+        auto guarantee_rhs = guarantee.arguments[1].literal();
+        if (!rhs || !guarantee_rhs) return expr;
+
+        if (!rhs->is_scalar() || !guarantee_rhs->is_scalar()) {
+          return expr;
+        }
+
+        ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_guarantee_rhs,
+                              Comparison::Execute(*rhs, *guarantee_rhs));
+        DCHECK_NE(cmp_rhs_guarantee_rhs, Comparison::NA);
+
+        if (cmp_rhs_guarantee_rhs == Comparison::EQUAL) {
+          // RHS of filter is equal to RHS of guarantee
+
+          if ((*cmp_guarantee & *cmp) == *cmp_guarantee) {
+            // guarantee is a subset of filter, so all data will be included
+            return literal(true);
+          }
+
+          if ((*cmp_guarantee & *cmp) == 0) {
+            // guarantee disjoint with filter, so all data will be excluded
+            return literal(false);
+          }
+
+          return expr;
+        }
+
+        if (*cmp_guarantee & cmp_rhs_guarantee_rhs) {
+          // unusable guarantee

Review comment:
       Give an example?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();
+
+  compute::KernelContext kernel_context(exec_context);
+  kernel_context.SetState(call->kernel_state.get());
+
+  auto kernel = call->kernel;
+  auto descrs = GetDescriptors(arguments);
+  auto options = call->options.get();
+  RETURN_NOT_OK(executor->Init(&kernel_context, {kernel, descrs, options}));
+
+  auto listener = std::make_shared<compute::detail::DatumAccumulator>();
+  RETURN_NOT_OK(executor->Execute(arguments, listener.get()));
+  return executor->WrapResults(arguments, listener->values());
+}
+
+std::array<std::pair<const Expression&, const Expression&>, 2>
+ArgumentsAndFlippedArguments(const Expression::Call& call) {
+  DCHECK_EQ(call.arguments.size(), 2);
+  return {std::pair<const Expression&, const Expression&>{call.arguments[0],
+                                                          call.arguments[1]},
+          std::pair<const Expression&, const Expression&>{call.arguments[1],
+                                                          call.arguments[0]}};
+}
+
+template <typename BinOp, typename It,
+          typename Out = typename std::iterator_traits<It>::value_type>
+util::optional<Out> FoldLeft(It begin, It end, const BinOp& bin_op) {
+  if (begin == end) return util::nullopt;
+
+  Out folded = std::move(*begin++);
+  while (begin != end) {
+    folded = bin_op(std::move(folded), std::move(*begin++));
+  }
+  return folded;
+}
+
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  if (call.function && call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
+  }
+  return util::nullopt;
+}
+
+bool DefinitelyNotNull(const Expression& expr) {
+  DCHECK(expr.IsBound());
+
+  if (expr.literal()) {
+    return !expr.IsNullLiteral();
+  }
+
+  if (expr.field_ref()) return false;
+
+  auto call = CallNotNull(expr);
+  if (auto null_handling = GetNullHandling(*call)) {
+    if (null_handling == compute::NullHandling::OUTPUT_NOT_NULL) {
+      return true;
+    }
+    if (null_handling == compute::NullHandling::INTERSECTION) {
+      return std::all_of(call->arguments.begin(), call->arguments.end(),
+                         DefinitelyNotNull);
+    }
+  }
+
+  return false;
+}
+
+std::vector<FieldRef> FieldsInExpression(const Expression& expr) {
+  if (auto lit = expr.literal()) return {};
+
+  if (auto ref = expr.field_ref()) {
+    return {*ref};
+  }
+
+  std::vector<FieldRef> fields;
+  for (const Expression& arg : CallNotNull(expr)->arguments) {
+    auto argument_fields = FieldsInExpression(arg);
+    std::move(argument_fields.begin(), argument_fields.end(), std::back_inserter(fields));
+  }
+  return fields;
+}
+
+Result<Expression> FoldConstants(Expression expr) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [](Expression expr, ...) -> Result<Expression> {
+        auto call = CallNotNull(expr);
+        if (std::all_of(call->arguments.begin(), call->arguments.end(),
+                        [](const Expression& argument) { return argument.literal(); })) {
+          // all arguments are literal; we can evaluate this subexpression *now*
+          static const Datum ignored_input;
+          ARROW_ASSIGN_OR_RAISE(Datum constant,
+                                ExecuteScalarExpression(expr, ignored_input));
+
+          return literal(std::move(constant));
+        }
+
+        // XXX the following should probably be in a registry of passes instead
+        // of inline
+
+        if (GetNullHandling(*call) == compute::NullHandling::INTERSECTION) {
+          // kernels which always produce intersected validity can be resolved
+          // to null *now* if any of their inputs is a null literal
+          for (const auto& argument : call->arguments) {
+            if (argument.IsNullLiteral()) {
+              return argument;
+            }
+          }
+        }
+
+        if (call->function_name == "and_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // true and x == x
+            if (args.first == literal(true)) return args.second;
+
+            // false and x == false
+            if (args.first == literal(false)) return args.first;
+
+            // x and x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        if (call->function_name == "or_kleene") {
+          for (auto args : ArgumentsAndFlippedArguments(*call)) {
+            // false or x == x
+            if (args.first == literal(false)) return args.second;
+
+            // true or x == true
+            if (args.first == literal(true)) return args.first;
+
+            // x or x == x
+            if (args.first == args.second) return args.first;
+          }
+          return expr;
+        }
+
+        return expr;
+      });
+}
+
+inline std::vector<Expression> GuaranteeConjunctionMembers(
+    const Expression& guaranteed_true_predicate) {
+  auto guarantee = guaranteed_true_predicate.call();
+  if (!guarantee || guarantee->function_name != "and_kleene") {
+    return {guaranteed_true_predicate};
+  }
+  return FlattenedAssociativeChain(guaranteed_true_predicate).fringe;
+}
+
+// Conjunction members which are represented in known_values are erased from
+// conjunction_members
+Status ExtractKnownFieldValuesImpl(
+    std::vector<Expression>* conjunction_members,
+    std::unordered_map<FieldRef, Datum, FieldRef::Hash>* known_values) {
+  auto unconsumed_end =
+      std::partition(conjunction_members->begin(), conjunction_members->end(),
+                     [](const Expression& expr) {
+                       // search for an equality conditions between a field and a literal
+                       auto call = expr.call();
+                       if (!call) return true;
+
+                       if (call->function_name == "equal") {
+                         auto ref = call->arguments[0].field_ref();
+                         auto lit = call->arguments[1].literal();
+                         return !(ref && lit);
+                       }
+
+                       return true;
+                     });
+
+  for (auto it = unconsumed_end; it != conjunction_members->end(); ++it) {
+    auto call = CallNotNull(*it);
+
+    auto ref = call->arguments[0].field_ref();
+    auto lit = call->arguments[1].literal();
+
+    auto it_success = known_values->emplace(*ref, *lit);
+    if (it_success.second) continue;
+
+    // A value was already known for ref; check it
+    auto ref_lit = it_success.first;
+    if (*lit != ref_lit->second) {
+      return Status::Invalid("Conflicting guarantees: (", ref->ToString(),
+                             " == ", lit->ToString(), ") vs (", ref->ToString(),
+                             " == ", ref_lit->second.ToString());
+    }
+  }
+
+  conjunction_members->erase(unconsumed_end, conjunction_members->end());
+
+  return Status::OK();
+}
+
+Result<std::unordered_map<FieldRef, Datum, FieldRef::Hash>> ExtractKnownFieldValues(
+    const Expression& guaranteed_true_predicate) {
+  auto conjunction_members = GuaranteeConjunctionMembers(guaranteed_true_predicate);
+  std::unordered_map<FieldRef, Datum, FieldRef::Hash> known_values;
+  RETURN_NOT_OK(ExtractKnownFieldValuesImpl(&conjunction_members, &known_values));
+  return known_values;
+}
+
+Result<Expression> ReplaceFieldsWithKnownValues(
+    const std::unordered_map<FieldRef, Datum, FieldRef::Hash>& known_values,
+    Expression expr) {
+  if (!expr.IsBound()) {
+    return Status::Invalid(
+        "ReplaceFieldsWithKnownValues called on an unbound Expression");
+  }
+
+  return Modify(
+      std::move(expr),
+      [&known_values](Expression expr) -> Result<Expression> {
+        if (auto ref = expr.field_ref()) {
+          auto it = known_values.find(*ref);
+          if (it != known_values.end()) {
+            ARROW_ASSIGN_OR_RAISE(Datum lit,
+                                  compute::Cast(it->second, expr.descr().type));
+            return literal(std::move(lit));
+          }
+        }
+        return expr;
+      },
+      [](Expression expr, ...) { return expr; });
+}
+
+inline bool IsBinaryAssociativeCommutative(const Expression::Call& call) {
+  static std::unordered_set<std::string> binary_associative_commutative{
+      "and",      "or",  "and_kleene",       "or_kleene",  "xor",
+      "multiply", "add", "multiply_checked", "add_checked"};
+
+  auto it = binary_associative_commutative.find(call.function_name);
+  return it != binary_associative_commutative.end();
+}
+
+Result<Expression> Canonicalize(Expression expr, compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Canonicalize(std::move(expr), &exec_context);
+  }
+
+  // If potentially reconstructing more deeply than a call's immediate arguments
+  // (for example, when reorganizing an associative chain), add expressions to this set to
+  // avoid unnecessary work
+  struct {
+    std::unordered_set<Expression, Expression::Hash> set_;
+
+    bool operator()(const Expression& expr) const {
+      return set_.find(expr) != set_.end();
+    }
+
+    void Add(std::vector<Expression> exprs) {
+      std::move(exprs.begin(), exprs.end(), std::inserter(set_, set_.end()));
+    }
+  } AlreadyCanonicalized;
+
+  return Modify(
+      std::move(expr),
+      [&AlreadyCanonicalized, exec_context](Expression expr) -> Result<Expression> {
+        auto call = expr.call();
+        if (!call) return expr;
+
+        if (AlreadyCanonicalized(expr)) return expr;
+
+        if (IsBinaryAssociativeCommutative(*call)) {
+          struct {
+            int Priority(const Expression& operand) const {
+              // order literals first, starting with nulls
+              if (operand.IsNullLiteral()) return 0;
+              if (operand.literal()) return 1;
+              return 2;
+            }
+            bool operator()(const Expression& l, const Expression& r) const {
+              return Priority(l) < Priority(r);
+            }
+          } CanonicalOrdering;
+
+          FlattenedAssociativeChain chain(expr);
+          if (chain.was_left_folded &&
+              std::is_sorted(chain.fringe.begin(), chain.fringe.end(),
+                             CanonicalOrdering)) {
+            AlreadyCanonicalized.Add(std::move(chain.exprs));
+            return expr;
+          }
+
+          std::stable_sort(chain.fringe.begin(), chain.fringe.end(), CanonicalOrdering);
+
+          // fold the chain back up
+          auto folded =
+              FoldLeft(chain.fringe.begin(), chain.fringe.end(),
+                       [call, &AlreadyCanonicalized](Expression l, Expression r) {
+                         auto canonicalized_call = *call;
+                         canonicalized_call.arguments = {std::move(l), std::move(r)};
+                         Expression expr(std::move(canonicalized_call));
+                         AlreadyCanonicalized.Add({expr});
+                         return expr;
+                       });
+          return std::move(*folded);
+        }
+
+        if (auto cmp = Comparison::Get(call->function_name)) {
+          if (call->arguments[0].literal() && !call->arguments[1].literal()) {
+            // ensure that literals are on comparisons' RHS
+            auto flipped_call = *call;
+            flipped_call.function_name =
+                Comparison::GetName(Comparison::GetFlipped(*cmp));
+            // look up the flipped kernel
+            // TODO extract a helper for use here and in Bind
+            ARROW_ASSIGN_OR_RAISE(
+                auto function,
+                exec_context->func_registry()->GetFunction(flipped_call.function_name));
+
+            auto descrs = GetDescriptors(flipped_call.arguments);
+            ARROW_ASSIGN_OR_RAISE(flipped_call.kernel, function->DispatchExact(descrs));
+
+            std::swap(flipped_call.arguments[0], flipped_call.arguments[1]);
+            return Expression(std::move(flipped_call));
+          }
+        }
+
+        return expr;
+      },
+      [](Expression expr, ...) { return expr; });
+}
+
+Result<Expression> DirectComparisonSimplification(Expression expr,
+                                                  const Expression::Call& guarantee) {
+  return Modify(
+      std::move(expr), [](Expression expr) { return expr; },
+      [&guarantee](Expression expr, ...) -> Result<Expression> {
+        auto call = expr.call();
+        if (!call) return expr;
+
+        // Ensure both calls are comparisons with equal LHS and scalar RHS
+        auto cmp = Comparison::Get(expr);
+        auto cmp_guarantee = Comparison::Get(guarantee.function_name);
+        if (!cmp || !cmp_guarantee) return expr;
+
+        if (call->arguments[0] != guarantee.arguments[0]) return expr;
+
+        auto rhs = call->arguments[1].literal();
+        auto guarantee_rhs = guarantee.arguments[1].literal();
+        if (!rhs || !guarantee_rhs) return expr;
+
+        if (!rhs->is_scalar() || !guarantee_rhs->is_scalar()) {
+          return expr;
+        }
+
+        ARROW_ASSIGN_OR_RAISE(auto cmp_rhs_guarantee_rhs,
+                              Comparison::Execute(*rhs, *guarantee_rhs));
+        DCHECK_NE(cmp_rhs_guarantee_rhs, Comparison::NA);
+
+        if (cmp_rhs_guarantee_rhs == Comparison::EQUAL) {
+          // RHS of filter is equal to RHS of guarantee
+
+          if ((*cmp_guarantee & *cmp) == *cmp_guarantee) {
+            // guarantee is a subset of filter, so all data will be included
+            return literal(true);
+          }
+
+          if ((*cmp_guarantee & *cmp) == 0) {
+            // guarantee disjoint with filter, so all data will be excluded
+            return literal(false);
+          }
+
+          return expr;
+        }
+
+        if (*cmp_guarantee & cmp_rhs_guarantee_rhs) {
+          // unusable guarantee
+          return expr;
+        }
+
+        if (*cmp & Comparison::GetFlipped(cmp_rhs_guarantee_rhs)) {
+          // x > 1, x >= 1, x != 1 guaranteed by x >= 3
+          return literal(true);
+        } else {
+          // x < 1, x <= 1, x == 1 unsatisfiable if x >= 3
+          return literal(false);
+        }
+      });
+}
+
+Result<Expression> SimplifyWithGuarantee(Expression expr,
+                                         const Expression& guaranteed_true_predicate) {
+  auto conjunction_members = GuaranteeConjunctionMembers(guaranteed_true_predicate);
+
+  std::unordered_map<FieldRef, Datum, FieldRef::Hash> known_values;
+  RETURN_NOT_OK(ExtractKnownFieldValuesImpl(&conjunction_members, &known_values));
+
+  ARROW_ASSIGN_OR_RAISE(expr,
+                        ReplaceFieldsWithKnownValues(known_values, std::move(expr)));

Review comment:
       Is this useful, given that `DirectComparisonSimplification` should catch these cases as well?

##########
File path: cpp/src/arrow/dataset/expression.cc
##########
@@ -0,0 +1,1177 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/dataset/expression.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/chunked_array.h"
+#include "arrow/compute/exec_internal.h"
+#include "arrow/dataset/expression_internal.h"
+#include "arrow/io/memory.h"
+#include "arrow/ipc/reader.h"
+#include "arrow/ipc/writer.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+namespace dataset {
+
+Expression::Expression(Call call) : impl_(std::make_shared<Impl>(std::move(call))) {}
+
+Expression::Expression(Datum literal)
+    : impl_(std::make_shared<Impl>(std::move(literal))) {}
+
+Expression::Expression(Parameter parameter)
+    : impl_(std::make_shared<Impl>(std::move(parameter))) {}
+
+Expression literal(Datum lit) { return Expression(std::move(lit)); }
+
+Expression field_ref(FieldRef ref) {
+  return Expression(Expression::Parameter{std::move(ref), {}});
+}
+
+Expression call(std::string function, std::vector<Expression> arguments,
+                std::shared_ptr<compute::FunctionOptions> options) {
+  Expression::Call call;
+  call.function_name = std::move(function);
+  call.arguments = std::move(arguments);
+  call.options = std::move(options);
+  return Expression(std::move(call));
+}
+
+const Datum* Expression::literal() const { return util::get_if<Datum>(impl_.get()); }
+
+const FieldRef* Expression::field_ref() const {
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return &parameter->ref;
+  }
+  return nullptr;
+}
+
+const Expression::Call* Expression::call() const {
+  return util::get_if<Call>(impl_.get());
+}
+
+ValueDescr Expression::descr() const {
+  if (impl_ == nullptr) return {};
+
+  if (auto lit = literal()) {
+    return lit->descr();
+  }
+
+  if (auto parameter = util::get_if<Parameter>(impl_.get())) {
+    return parameter->descr;
+  }
+
+  return CallNotNull(*this)->descr;
+}
+
+std::string Expression::ToString() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      switch (lit->type()->id()) {
+        case Type::STRING:
+        case Type::LARGE_STRING:
+          return '"' +
+                 Escape(util::string_view(*lit->scalar_as<BaseBinaryScalar>().value)) +
+                 '"';
+
+        case Type::BINARY:
+        case Type::FIXED_SIZE_BINARY:
+        case Type::LARGE_BINARY:
+          return '"' + lit->scalar_as<BaseBinaryScalar>().value->ToHexString() + '"';
+
+        default:
+          break;
+      }
+      return lit->scalar()->ToString();
+    }
+    return lit->ToString();
+  }
+
+  if (auto ref = field_ref()) {
+    if (auto name = ref->name()) {
+      return *name;
+    }
+    if (auto path = ref->field_path()) {
+      return path->ToString();
+    }
+    return ref->ToString();
+  }
+
+  auto call = CallNotNull(*this);
+  auto binary = [&](std::string op) {
+    return "(" + call->arguments[0].ToString() + " " + op + " " +
+           call->arguments[1].ToString() + ")";
+  };
+
+  if (auto cmp = Comparison::Get(call->function_name)) {
+    return binary(Comparison::GetOp(*cmp));
+  }
+
+  constexpr util::string_view kleene = "_kleene";
+  if (util::string_view{call->function_name}.ends_with(kleene)) {
+    auto op = call->function_name.substr(0, call->function_name.size() - kleene.size());
+    return binary(std::move(op));
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    std::string out = "{";
+    auto argument = call->arguments.begin();
+    for (const auto& field_name : options->field_names) {
+      out += field_name + "=" + argument++->ToString() + ", ";
+    }
+    out.resize(out.size() - 1);
+    out.back() = '}';
+    return out;
+  }
+
+  std::string out = call->function_name + "(";
+  for (const auto& arg : call->arguments) {
+    out += arg.ToString() + ", ";
+  }
+
+  if (call->options == nullptr) {
+    out.resize(out.size() - 1);
+    out.back() = ')';
+    return out;
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    DCHECK_EQ(options->value_set.kind(), Datum::ARRAY);
+    out += "value_set=" + options->value_set.make_array()->ToString();
+    if (options->skip_nulls) {
+      out += ", skip_nulls";
+    }
+    return out + ")";
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    if (options->to_type == nullptr) {
+      return out + "to_type=<INVALID NOT PROVIDED>)";
+    }
+    out += "to_type=" + options->to_type->ToString();
+    if (options->allow_int_overflow) out += ", allow_int_overflow";
+    if (options->allow_time_truncate) out += ", allow_time_truncate";
+    if (options->allow_time_overflow) out += ", allow_time_overflow";
+    if (options->allow_decimal_truncate) out += ", allow_decimal_truncate";
+    if (options->allow_float_truncate) out += ", allow_float_truncate";
+    if (options->allow_invalid_utf8) out += ", allow_invalid_utf8";
+    return out + ")";
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    return out + "format=" + options->format +
+           ", unit=" + internal::ToString(options->unit) + ")";
+  }
+
+  return out + "{NON-REPRESENTABLE OPTIONS})";
+}
+
+void PrintTo(const Expression& expr, std::ostream* os) {
+  *os << expr.ToString();
+  if (expr.IsBound()) {
+    *os << "[bound]";
+  }
+}
+
+bool Expression::Equals(const Expression& other) const {
+  if (Identical(*this, other)) return true;
+
+  if (impl_->index() != other.impl_->index()) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    return lit->Equals(*other.literal());
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->Equals(*other.field_ref());
+  }
+
+  auto call = CallNotNull(*this);
+  auto other_call = CallNotNull(other);
+
+  if (call->function_name != other_call->function_name ||
+      call->kernel != other_call->kernel) {
+    return false;
+  }
+
+  for (size_t i = 0; i < call->arguments.size(); ++i) {
+    if (!call->arguments[i].Equals(other_call->arguments[i])) {
+      return false;
+    }
+  }
+
+  if (call->options == other_call->options) return true;
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    auto other_options = GetSetLookupOptions(*other_call);
+    return options->value_set == other_options->value_set &&
+           options->skip_nulls == other_options->skip_nulls;
+  }
+
+  if (auto options = GetCastOptions(*call)) {
+    auto other_options = GetCastOptions(*other_call);
+    for (auto safety_opt : {
+             &compute::CastOptions::allow_int_overflow,
+             &compute::CastOptions::allow_time_truncate,
+             &compute::CastOptions::allow_time_overflow,
+             &compute::CastOptions::allow_decimal_truncate,
+             &compute::CastOptions::allow_float_truncate,
+             &compute::CastOptions::allow_invalid_utf8,
+         }) {
+      if (options->*safety_opt != other_options->*safety_opt) return false;
+    }
+    return options->to_type->Equals(other_options->to_type);
+  }
+
+  if (auto options = GetStructOptions(*call)) {
+    auto other_options = GetStructOptions(*other_call);
+    return options->field_names == other_options->field_names;
+  }
+
+  if (auto options = GetStrptimeOptions(*call)) {
+    auto other_options = GetStrptimeOptions(*other_call);
+    return options->format == other_options->format &&
+           options->unit == other_options->unit;
+  }
+
+  ARROW_LOG(WARNING) << "comparing unknown FunctionOptions for function "
+                     << call->function_name;
+  return false;
+}
+
+size_t Expression::hash() const {
+  if (auto lit = literal()) {
+    if (lit->is_scalar()) {
+      return Scalar::Hash::hash(*lit->scalar());
+    }
+    return 0;
+  }
+
+  if (auto ref = field_ref()) {
+    return ref->hash();
+  }
+
+  auto call = CallNotNull(*this);
+
+  size_t out = std::hash<std::string>{}(call->function_name);
+  for (const auto& arg : call->arguments) {
+    out ^= arg.hash();
+  }
+  return out;
+}
+
+bool Expression::IsBound() const {
+  if (descr().type == nullptr) return false;
+
+  if (auto lit = literal()) return true;
+
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsBound()) return false;
+  }
+
+  return call->kernel != nullptr;
+}
+
+bool Expression::IsScalarExpression() const {
+  if (auto lit = literal()) {
+    return lit->is_scalar();
+  }
+
+  // FIXME handle case where a list's item field is referenced
+  if (auto ref = field_ref()) return true;
+
+  auto call = CallNotNull(*this);
+
+  for (const Expression& arg : call->arguments) {
+    if (!arg.IsScalarExpression()) return false;
+  }
+
+  if (call->function) {
+    return call->function->kind() == compute::Function::SCALAR;
+  }
+
+  // this expression is not bound; make a best guess based on
+  // the default function registry
+  if (auto function = compute::GetFunctionRegistry()
+                          ->GetFunction(call->function_name)
+                          .ValueOr(nullptr)) {
+    return function->kind() == compute::Function::SCALAR;
+  }
+
+  // unknown function or other error; conservatively return false
+  return false;
+}
+
+bool Expression::IsNullLiteral() const {
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
+bool Expression::IsSatisfiable() const {
+  if (descr().type && descr().type->id() == Type::NA) {
+    return false;
+  }
+
+  if (auto lit = literal()) {
+    if (lit->null_count() == lit->length()) {
+      return false;
+    }
+
+    if (lit->is_scalar() && lit->type()->id() == Type::BOOL) {
+      return lit->scalar_as<BooleanScalar>().value;
+    }
+  }
+
+  if (auto ref = field_ref()) {
+    return true;
+  }
+
+  return true;
+}
+
+inline bool KernelStateIsImmutable(const std::string& function) {
+  // XXX maybe just add Kernel::state_is_immutable or so?
+
+  // known functions with non-null but nevertheless immutable KernelState
+  static std::unordered_set<std::string> names = {
+      "is_in", "index_in", "cast", "struct", "strptime",
+  };
+
+  return names.find(function) != names.end();
+}
+
+Result<std::unique_ptr<compute::KernelState>> InitKernelState(
+    const Expression::Call& call, compute::ExecContext* exec_context) {
+  if (!call.kernel->init) return nullptr;
+
+  compute::KernelContext kernel_context(exec_context);
+  auto kernel_state = call.kernel->init(
+      &kernel_context, {call.kernel, GetDescriptors(call.arguments), call.options.get()});
+
+  RETURN_NOT_OK(kernel_context.status());
+  return std::move(kernel_state);
+}
+
+Status MaybeInsertCast(std::shared_ptr<DataType> to_type, Expression* expr) {
+  if (expr->descr().type->Equals(to_type)) {
+    return Status::OK();
+  }
+
+  if (auto lit = expr->literal()) {
+    ARROW_ASSIGN_OR_RAISE(Datum new_lit, compute::Cast(*lit, to_type));
+    *expr = literal(std::move(new_lit));
+    return Status::OK();
+  }
+
+  // FIXME the resulting cast Call must be bound but this is a hack
+  auto with_cast = call("cast", {literal(MakeNullScalar(expr->descr().type))},
+                        compute::CastOptions::Safe(to_type));
+
+  static ValueDescr ignored_descr;
+  ARROW_ASSIGN_OR_RAISE(with_cast, with_cast.Bind(ignored_descr));
+
+  auto call_with_cast = *CallNotNull(with_cast);
+  call_with_cast.arguments[0] = std::move(*expr);
+  call_with_cast.descr = ValueDescr{std::move(to_type), expr->descr().shape};
+
+  *expr = Expression(std::move(call_with_cast));
+  return Status::OK();
+}
+
+Status InsertImplicitCasts(Expression::Call* call) {
+  DCHECK(std::all_of(call->arguments.begin(), call->arguments.end(),
+                     [](const Expression& argument) { return argument.IsBound(); }));
+
+  if (IsSameTypesBinary(call->function_name)) {
+    for (auto&& argument : call->arguments) {
+      if (auto value_type = GetDictionaryValueType(argument.descr().type)) {
+        RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &argument));
+      }
+    }
+
+    if (call->arguments[0].descr().shape == ValueDescr::SCALAR) {
+      // argument 0 is scalar so casting is cheap
+      return MaybeInsertCast(call->arguments[1].descr().type, &call->arguments[0]);
+    }
+
+    // cast argument 1 unconditionally
+    return MaybeInsertCast(call->arguments[0].descr().type, &call->arguments[1]);
+  }
+
+  if (auto options = GetSetLookupOptions(*call)) {
+    if (auto value_type = GetDictionaryValueType(call->arguments[0].descr().type)) {
+      // DICTIONARY input is not supported; decode it.
+      RETURN_NOT_OK(MaybeInsertCast(std::move(value_type), &call->arguments[0]));
+    }
+
+    if (options->value_set.type()->id() == Type::DICTIONARY) {
+      // DICTIONARY value_set is not supported; decode it.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      RETURN_NOT_OK(EnsureNotDictionary(&new_options->value_set));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    if (!options->value_set.type()->Equals(call->arguments[0].descr().type)) {
+      // The value_set is assumed smaller than inputs, casting it should be cheaper.
+      auto new_options = std::make_shared<compute::SetLookupOptions>(*options);
+      ARROW_ASSIGN_OR_RAISE(new_options->value_set,
+                            compute::Cast(std::move(new_options->value_set),
+                                          call->arguments[0].descr().type));
+      options = new_options.get();
+      call->options = std::move(new_options);
+    }
+
+    return Status::OK();
+  }
+
+  return Status::OK();
+}
+
+Result<Expression> Expression::Bind(ValueDescr in,
+                                    compute::ExecContext* exec_context) const {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return Bind(std::move(in), &exec_context);
+  }
+
+  if (literal()) return *this;
+
+  if (auto ref = field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(auto field, ref->GetOneOrNone(*in.type));
+    auto descr = field ? ValueDescr{field->type(), in.shape} : ValueDescr::Scalar(null());
+    return Expression{Parameter{*ref, std::move(descr)}};
+  }
+
+  auto bound_call = *CallNotNull(*this);
+
+  ARROW_ASSIGN_OR_RAISE(bound_call.function, GetFunction(bound_call, exec_context));
+
+  for (auto&& argument : bound_call.arguments) {
+    ARROW_ASSIGN_OR_RAISE(argument, argument.Bind(in, exec_context));
+  }
+  RETURN_NOT_OK(InsertImplicitCasts(&bound_call));
+
+  auto descrs = GetDescriptors(bound_call.arguments);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel, bound_call.function->DispatchExact(descrs));
+
+  compute::KernelContext kernel_context(exec_context);
+  ARROW_ASSIGN_OR_RAISE(bound_call.kernel_state,
+                        InitKernelState(bound_call, exec_context));
+  kernel_context.SetState(bound_call.kernel_state.get());
+
+  ARROW_ASSIGN_OR_RAISE(
+      bound_call.descr,
+      bound_call.kernel->signature->out_type().Resolve(&kernel_context, descrs));
+
+  return Expression(std::move(bound_call));
+}
+
+Result<Expression> Expression::Bind(const Schema& in_schema,
+                                    compute::ExecContext* exec_context) const {
+  return Bind(ValueDescr::Array(struct_(in_schema.fields())), exec_context);
+}
+
+Result<Datum> ExecuteScalarExpression(const Expression& expr, const Datum& input,
+                                      compute::ExecContext* exec_context) {
+  if (exec_context == nullptr) {
+    compute::ExecContext exec_context;
+    return ExecuteScalarExpression(expr, input, &exec_context);
+  }
+
+  if (!expr.IsBound()) {
+    return Status::Invalid("Cannot Execute unbound expression.");
+  }
+
+  if (!expr.IsScalarExpression()) {
+    return Status::Invalid(
+        "ExecuteScalarExpression cannot Execute non-scalar expression ", expr.ToString());
+  }
+
+  if (auto lit = expr.literal()) return *lit;
+
+  if (auto ref = expr.field_ref()) {
+    ARROW_ASSIGN_OR_RAISE(Datum field, GetDatumField(*ref, input));
+
+    if (field.descr() != expr.descr()) {
+      // Refernced field was present but didn't have the expected type.
+      // Should we just error here? For now, pay dispatch cost and just cast.
+      ARROW_ASSIGN_OR_RAISE(
+          field, compute::Cast(field, expr.descr().type, compute::CastOptions::Safe(),
+                               exec_context));
+    }
+
+    return field;
+  }
+
+  auto call = CallNotNull(expr);
+
+  std::vector<Datum> arguments(call->arguments.size());
+  for (size_t i = 0; i < arguments.size(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(
+        arguments[i], ExecuteScalarExpression(call->arguments[i], input, exec_context));
+  }
+
+  auto executor = compute::detail::KernelExecutor::MakeScalar();
+
+  compute::KernelContext kernel_context(exec_context);
+  kernel_context.SetState(call->kernel_state.get());
+
+  auto kernel = call->kernel;
+  auto descrs = GetDescriptors(arguments);
+  auto options = call->options.get();
+  RETURN_NOT_OK(executor->Init(&kernel_context, {kernel, descrs, options}));
+
+  auto listener = std::make_shared<compute::detail::DatumAccumulator>();
+  RETURN_NOT_OK(executor->Execute(arguments, listener.get()));
+  return executor->WrapResults(arguments, listener->values());
+}
+
+std::array<std::pair<const Expression&, const Expression&>, 2>
+ArgumentsAndFlippedArguments(const Expression::Call& call) {
+  DCHECK_EQ(call.arguments.size(), 2);
+  return {std::pair<const Expression&, const Expression&>{call.arguments[0],
+                                                          call.arguments[1]},
+          std::pair<const Expression&, const Expression&>{call.arguments[1],
+                                                          call.arguments[0]}};
+}
+
+template <typename BinOp, typename It,
+          typename Out = typename std::iterator_traits<It>::value_type>
+util::optional<Out> FoldLeft(It begin, It end, const BinOp& bin_op) {
+  if (begin == end) return util::nullopt;
+
+  Out folded = std::move(*begin++);
+  while (begin != end) {
+    folded = bin_op(std::move(folded), std::move(*begin++));
+  }
+  return folded;
+}
+
+util::optional<compute::NullHandling::type> GetNullHandling(
+    const Expression::Call& call) {
+  if (call.function && call.function->kind() == compute::Function::SCALAR) {
+    return static_cast<const compute::ScalarKernel*>(call.kernel)->null_handling;
+  }
+  return util::nullopt;
+}
+
+bool DefinitelyNotNull(const Expression& expr) {
+  DCHECK(expr.IsBound());
+
+  if (expr.literal()) {
+    return !expr.IsNullLiteral();
+  }
+
+  if (expr.field_ref()) return false;

Review comment:
       Hmm... what if the referenced field has `nullable == false`?




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