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

[GitHub] [arrow] westonpace commented on a change in pull request #12279: ARROW-15238: [C++] ARROW_ENGINE module with substrait consumer

westonpace commented on a change in pull request #12279:
URL: https://github.com/apache/arrow/pull/12279#discussion_r796032104



##########
File path: cpp/cmake_modules/DefineOptions.cmake
##########
@@ -478,6 +478,13 @@ advised that if this is enabled 'install' will fail silently on components;\
 that have not been built"
                 OFF)
 
+  set(ARROW_SUBSTRAIT_REPO_AND_TAG_DEFAULT
+      "https://github.com/substrait-io/substrait e1b4c04a1b518912f4c4065b16a1b2c0ac8e14cf"
+  )
+  define_option_string(ARROW_SUBSTRAIT_REPO_AND_TAG
+                       "Custom 'repository_url tag' for generating substrait accessors"
+                       "${ARROW_SUBSTRAIT_REPO_AND_TAG_DEFAULT}")
+

Review comment:
       Would this be better as two options?  I'd face a little anxiety setting this as there is a space in the option.  Also, I'm not sure what "generating substrait accessors" means.  Can we say something like "for downloading substrait source"?

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;
+};
+
+int main(int argc, char** argv) {
+  // Plans arrive at the consumer serialized in a substrait-formatted Buffer
+  auto maybe_serialized_plan = GetSubstraitFromServer().result();
+  ABORT_ON_FAILURE(maybe_serialized_plan.status());
+  std::shared_ptr<arrow::Buffer> serialized_plan =
+      std::move(maybe_serialized_plan).ValueOrDie();
+
+  // Print the received plan to stdout as JSON
+  arrow::Result<std::string> maybe_plan_json =
+      eng::internal::SubstraitToJSON("Plan", *serialized_plan);
+  ABORT_ON_FAILURE(maybe_plan_json.status());
+  std::cout << std::string('#', 50) << " received substrait::Plan:" << std::endl;
+  std::cout << maybe_plan_json.ValueOrDie() << std::endl;
+
+  // Deserializing a plan requires a factory for consumers: each time a sink node is
+  // deserialized, a consumer is constructed into which its batches will be piped.
+  std::vector<std::shared_ptr<cp::SinkNodeConsumer>> consumers;
+  std::function<std::shared_ptr<cp::SinkNodeConsumer>()> consumer_factory = [&] {
+    // All batches produced by the plan will be fed into IgnoringConsumers:
+    auto tag = consumers.size();
+    consumers.emplace_back(new IgnoringConsumer{tag});
+    return consumers.back();
+  };
+
+  // NOTE Although most of the Deserialize functions require a const ExtensionSet& to
+  // resolve extension references, a Plan is what we use to construct that ExtensionSet.
+  // (It should be an optional output later.) In particular, it does not need to be kept
+  // alive nor does the serialized plan- none of the arrow:: objects in the output will
+  // contain references to memory owned by either.

Review comment:
       This is something of an unsatisfactory explanation for me.
   
   First of all, I think many users of this example will be of the mindset: "I have a thingy that generates substrait plans and I want to resolve those plans into data".  These users don't actually know the details of substrait and they have no idea what an extension set could possible even be referring to.
   
   Honestly, even users who are roughly familiar with substrait as a protocol (but haven't had to write a producer or consumer) probably gloss over all the complex details of extension sets.
   
   Since this is an introductory example, and `extension_set_out` is optional, you could omit the discussion entirely, especially since we already call out that it is safe to drop the serialized_plan.
   

##########
File path: cpp/src/arrow/engine/simple_extension_type_internal.h
##########
@@ -0,0 +1,183 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include "arrow/extension_type.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/reflection_internal.h"
+#include "arrow/util/string.h"
+
+namespace arrow {
+namespace engine {
+
+template <const util::string_view& kExtensionName, typename Params,

Review comment:
       This is a very nifty bit of metaprogramming however it is going to be hard to understand at a glance.  Can we document this helper?
   
   For example, it appears that params can be an arbitrary struct but must it be a flat struct?  We should probably mention that we are encoding it as a simple pseduo-JSON object (I say pseduo-JSON because any string properties won't have their enclosing double quotes and it will encode booleans as numbers).  Also, at some point down the road, we will need to create Substrait YAML to publish for all these extension types.  In addition, these should probably all be candidates for @rok 's "well-known extension types" once that is off the ground.
   
   Also, what kind of data types can I put into my struct?  From a naive glance maybe anything that supports `std::stringstream >> value`

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?
+#
+# Ultimately these declarations are a promise which needs to be backed by
+# equivalent serde in c++. For example, consider u8: when serializing to
+# substrait, we need to wrap instances of arrow::uint8 into the type
+# variation listed below. It would be ideal if we could SinglePointOfTruth
+# this correspondence; either generating c++ from the YAML or YAML from the
+# c++.
+#
+# At present (AFAICT) it's not valid to make this user extensible because
+# even if a user adds their custom scalar function to the registry *and*
+# defines the mapping from that scalar function to a substrait::ExtensionFunction
+# the corresponding YAML doesn't exist at any URI and so it can't be used in
+# substrait. Perhaps we could still help that case by providing a tool to
+# generate YAML from functions; that'd simplify the lives of people trying to
+# write arrow::compute::Functions to "define the function and if you want to
+# reference it from substrait generate this YAML and put it at some URI".
+#
+# In any case for the foreseeable future generation would be far too brittle;
+# URIs will not be accessed by anything but humans and the YAML is effectively
+# structured documentation. Thus extenders should pass the URI in the same way
+# they pass a description string; it's opaque to anything in arrow.
+#
+# We'll need a global mapping which contains:
+#  arrow::uint8 <-> {
+#   uri: "https://github.com/apache/arrow/blob//format/substrait/extension_types.yaml",
+#   name: "u8"
+#  }
+#
+# And additionally convenient builders for subsets of that mapping during serde:
+#  arrow::uint8 <-> reference to anchor
+#
+# Worth noting: it's valid to have multiple ExtensionFunctions with the same name,
+# as long as all impls' mangled ("compound") names are unique.
+# Also worth noting: Options may only be enumerations. This will get strange with
+# functions like is_in, where we'll have to require that one argument is a literal
+# list or something.
+
+type_variations:
+  - parent: i8
+    name: u8
+    description: an unsigned 8 bit integer
+    functions: SEPARATE
+  - parent: i16
+    name: u16
+    description: an unsigned 16 bit integer
+    functions: SEPARATE
+  - parent: i32
+    name: u32
+    description: an unsigned 32 bit integer
+    functions: SEPARATE
+  - parent: i32
+    name: u32
+    description: an unsigned 32 bit integer
+    functions: SEPARATE

Review comment:
       ```suggestion
   ```
   Looks like a duplicate entry

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"

Review comment:
       ```suggestion
   ```

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();

Review comment:
       Can we put the main method at the bottom of the file so we don't need this?

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;
+};
+
+int main(int argc, char** argv) {
+  // Plans arrive at the consumer serialized in a substrait-formatted Buffer
+  auto maybe_serialized_plan = GetSubstraitFromServer().result();
+  ABORT_ON_FAILURE(maybe_serialized_plan.status());
+  std::shared_ptr<arrow::Buffer> serialized_plan =
+      std::move(maybe_serialized_plan).ValueOrDie();
+
+  // Print the received plan to stdout as JSON
+  arrow::Result<std::string> maybe_plan_json =
+      eng::internal::SubstraitToJSON("Plan", *serialized_plan);
+  ABORT_ON_FAILURE(maybe_plan_json.status());
+  std::cout << std::string('#', 50) << " received substrait::Plan:" << std::endl;
+  std::cout << maybe_plan_json.ValueOrDie() << std::endl;
+
+  // Deserializing a plan requires a factory for consumers: each time a sink node is
+  // deserialized, a consumer is constructed into which its batches will be piped.
+  std::vector<std::shared_ptr<cp::SinkNodeConsumer>> consumers;
+  std::function<std::shared_ptr<cp::SinkNodeConsumer>()> consumer_factory = [&] {
+    // All batches produced by the plan will be fed into IgnoringConsumers:
+    auto tag = consumers.size();
+    consumers.emplace_back(new IgnoringConsumer{tag});
+    return consumers.back();
+  };
+
+  // NOTE Although most of the Deserialize functions require a const ExtensionSet& to
+  // resolve extension references, a Plan is what we use to construct that ExtensionSet.
+  // (It should be an optional output later.) In particular, it does not need to be kept
+  // alive nor does the serialized plan- none of the arrow:: objects in the output will
+  // contain references to memory owned by either.
+  auto maybe_decls = eng::DeserializePlan(*serialized_plan, consumer_factory);
+  ABORT_ON_FAILURE(maybe_decls.status());
+  std::vector<cp::Declaration> decls = std::move(maybe_decls).ValueOrDie();
+
+  // It's safe to drop the serialized plan; we don't leave references to its memory
+  serialized_plan.reset();
+
+  // Construct an empty plan (note: configure Function registry and ThreadPool here)
+  auto maybe_plan = cp::ExecPlan::Make();
+  ABORT_ON_FAILURE(maybe_plan.status());
+  std::shared_ptr<cp::ExecPlan> plan = std::move(maybe_plan).ValueOrDie();
+
+  for (const cp::Declaration& decl : decls) {
+    // Add decl to plan (note: configure ExecNode registry here)
+    ABORT_ON_FAILURE(decl.AddToPlan(plan.get()).status());
+  }
+
+  // Validate the plan and print it to stdout
+  ABORT_ON_FAILURE(plan->Validate());
+  std::cout << std::string('#', 50) << " produced arrow::ExecPlan:" << std::endl;
+  std::cout << plan->ToString() << std::endl;
+
+  // Start the plan...
+  std::cout << std::string('#', 50) << " consuming batches:" << std::endl;
+  ABORT_ON_FAILURE(plan->StartProducing());
+
+  // ... and wait for it to finish
+  ABORT_ON_FAILURE(plan->finished().status());
+  return EXIT_SUCCESS;
+}
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer() {

Review comment:
       I don't see a sink?  How is this going to work?  Or does each relation have an implicit sink?
   
   If it is the latter (implicit sink) I feel we should mention that in the example when we are talking about the factory of consumers (and one could even name it `RelationConsumer` instead of `SinkNodeConsumer` maybe to avoid the overloaded terminology)

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;

Review comment:
       Can we document this a little, "A unique label for instances to help distinguish logging output if a plan has multiple sinks"

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;
+};
+
+int main(int argc, char** argv) {
+  // Plans arrive at the consumer serialized in a substrait-formatted Buffer
+  auto maybe_serialized_plan = GetSubstraitFromServer().result();
+  ABORT_ON_FAILURE(maybe_serialized_plan.status());
+  std::shared_ptr<arrow::Buffer> serialized_plan =
+      std::move(maybe_serialized_plan).ValueOrDie();
+
+  // Print the received plan to stdout as JSON
+  arrow::Result<std::string> maybe_plan_json =
+      eng::internal::SubstraitToJSON("Plan", *serialized_plan);
+  ABORT_ON_FAILURE(maybe_plan_json.status());
+  std::cout << std::string('#', 50) << " received substrait::Plan:" << std::endl;
+  std::cout << maybe_plan_json.ValueOrDie() << std::endl;
+
+  // Deserializing a plan requires a factory for consumers: each time a sink node is
+  // deserialized, a consumer is constructed into which its batches will be piped.
+  std::vector<std::shared_ptr<cp::SinkNodeConsumer>> consumers;
+  std::function<std::shared_ptr<cp::SinkNodeConsumer>()> consumer_factory = [&] {
+    // All batches produced by the plan will be fed into IgnoringConsumers:
+    auto tag = consumers.size();
+    consumers.emplace_back(new IgnoringConsumer{tag});
+    return consumers.back();
+  };
+
+  // NOTE Although most of the Deserialize functions require a const ExtensionSet& to
+  // resolve extension references, a Plan is what we use to construct that ExtensionSet.
+  // (It should be an optional output later.) In particular, it does not need to be kept
+  // alive nor does the serialized plan- none of the arrow:: objects in the output will
+  // contain references to memory owned by either.
+  auto maybe_decls = eng::DeserializePlan(*serialized_plan, consumer_factory);

Review comment:
       For a code example we should avoid `auto` when calling our own methods as I think it is possible people may be scanning these without an IDE.

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;
+};
+
+int main(int argc, char** argv) {
+  // Plans arrive at the consumer serialized in a substrait-formatted Buffer
+  auto maybe_serialized_plan = GetSubstraitFromServer().result();
+  ABORT_ON_FAILURE(maybe_serialized_plan.status());
+  std::shared_ptr<arrow::Buffer> serialized_plan =
+      std::move(maybe_serialized_plan).ValueOrDie();

Review comment:
       One bonus of using `DoMain` is we can write all this as:
   
   ```
   ARROW_ASSIGN_OR_RAISE(std::shared_ptr<arrow::Buffer> serialized_plan, GetSubstraitFromServer());
   ```

##########
File path: cpp/cmake_modules/DefineOptions.cmake
##########
@@ -225,7 +225,7 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}")
 
   define_option(ARROW_DATASET "Build the Arrow Dataset Modules" OFF)
 
-  define_option(ARROW_ENGINE "Build the Arrow Execution Engine" OFF)
+  define_option(ARROW_ENGINE "Build the Arrow Query Engine Module" OFF)

Review comment:
       At the risk of bike shedding on names (apologies if this was in the previous PR), I have been liking the old name of "streaming execution engine".  Given that this module won't really process traditional "queries" directly (e.g. SQL) the name "execution engine" might help to distinguish things like "there is no planner" and "you can't speak SQL to this".

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;
+};
+
+int main(int argc, char** argv) {
+  // Plans arrive at the consumer serialized in a substrait-formatted Buffer
+  auto maybe_serialized_plan = GetSubstraitFromServer().result();
+  ABORT_ON_FAILURE(maybe_serialized_plan.status());
+  std::shared_ptr<arrow::Buffer> serialized_plan =
+      std::move(maybe_serialized_plan).ValueOrDie();
+
+  // Print the received plan to stdout as JSON
+  arrow::Result<std::string> maybe_plan_json =
+      eng::internal::SubstraitToJSON("Plan", *serialized_plan);
+  ABORT_ON_FAILURE(maybe_plan_json.status());
+  std::cout << std::string('#', 50) << " received substrait::Plan:" << std::endl;
+  std::cout << maybe_plan_json.ValueOrDie() << std::endl;
+
+  // Deserializing a plan requires a factory for consumers: each time a sink node is
+  // deserialized, a consumer is constructed into which its batches will be piped.
+  std::vector<std::shared_ptr<cp::SinkNodeConsumer>> consumers;
+  std::function<std::shared_ptr<cp::SinkNodeConsumer>()> consumer_factory = [&] {
+    // All batches produced by the plan will be fed into IgnoringConsumers:
+    auto tag = consumers.size();
+    consumers.emplace_back(new IgnoringConsumer{tag});
+    return consumers.back();
+  };
+
+  // NOTE Although most of the Deserialize functions require a const ExtensionSet& to
+  // resolve extension references, a Plan is what we use to construct that ExtensionSet.
+  // (It should be an optional output later.) In particular, it does not need to be kept
+  // alive nor does the serialized plan- none of the arrow:: objects in the output will
+  // contain references to memory owned by either.
+  auto maybe_decls = eng::DeserializePlan(*serialized_plan, consumer_factory);
+  ABORT_ON_FAILURE(maybe_decls.status());
+  std::vector<cp::Declaration> decls = std::move(maybe_decls).ValueOrDie();
+
+  // It's safe to drop the serialized plan; we don't leave references to its memory
+  serialized_plan.reset();
+
+  // Construct an empty plan (note: configure Function registry and ThreadPool here)

Review comment:
       It is a rather unfortunate (though maybe unavoidable) fact that both Substrait and Arrow have something called a "plan" but a Substrait "plan" does not deserialize into an Arrow "plan".  This "create an empty plan" step feels a little bit silly for the common use case.  Can we create two deserialization methods?
   
   `eng::DeserializePlan(...)` returns `Result<cp::ExecPlan>` while `eng::DeserializePlanAsFragment` returns `Result<std::vector<cp::Declaration>>`?

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {

Review comment:
       Minor nit: Another potential bike shedding topic but it just seems weird that:
   
   In order to use the "consumer" we need to create "consumers".

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;

Review comment:
       ```suggestion
   namespace eng = arrow::engine;
   namespace cp = arrow::compute;
   ```
   I'm pretty sure we can get away without the leading colons.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}

Review comment:
       Would it be more appropriate to add a constructor in `scalar.h`:
   
   ```
   FixedSizeBinaryScalar::FixedSizeBinaryScalar(const std::string&);
   ```

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;
+};
+
+int main(int argc, char** argv) {
+  // Plans arrive at the consumer serialized in a substrait-formatted Buffer

Review comment:
       Minor nit: "substrait-formatted buffer" is a little vague.  Maybe something like, "The consumer accepts Substrait plans expressed with protobuf serialization.  The consumer accepts both the JSON protobuf format and the binary protobuf format."

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \

Review comment:
       Minor nit: When writing examples, rather than introducing a new macro I try and write my main method like this...
   
   ```
   arrow::Status DoMain() {
     /// ...
   }
   
   int main() {
     arrow::Status status = DoMain();
     if (!status.ok()) {
       std::cerr << "An error occurred: " << status.message() << std::endl;
       return EXIT_FAILURE;
     } else {
       return EXIT_SUCCESS;
     }
   }
   ```
   
   That being said, the abort has the advantage of informing a bit more precisely where the error happened.  I'm probably fine either way.

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>

Review comment:
       ```suggestion
   #include <arrow/engine/api.h>
   ```

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;
+};
+
+int main(int argc, char** argv) {
+  // Plans arrive at the consumer serialized in a substrait-formatted Buffer
+  auto maybe_serialized_plan = GetSubstraitFromServer().result();
+  ABORT_ON_FAILURE(maybe_serialized_plan.status());
+  std::shared_ptr<arrow::Buffer> serialized_plan =
+      std::move(maybe_serialized_plan).ValueOrDie();
+
+  // Print the received plan to stdout as JSON
+  arrow::Result<std::string> maybe_plan_json =
+      eng::internal::SubstraitToJSON("Plan", *serialized_plan);
+  ABORT_ON_FAILURE(maybe_plan_json.status());
+  std::cout << std::string('#', 50) << " received substrait::Plan:" << std::endl;
+  std::cout << maybe_plan_json.ValueOrDie() << std::endl;
+
+  // Deserializing a plan requires a factory for consumers: each time a sink node is
+  // deserialized, a consumer is constructed into which its batches will be piped.
+  std::vector<std::shared_ptr<cp::SinkNodeConsumer>> consumers;
+  std::function<std::shared_ptr<cp::SinkNodeConsumer>()> consumer_factory = [&] {
+    // All batches produced by the plan will be fed into IgnoringConsumers:
+    auto tag = consumers.size();
+    consumers.emplace_back(new IgnoringConsumer{tag});
+    return consumers.back();
+  };
+
+  // NOTE Although most of the Deserialize functions require a const ExtensionSet& to
+  // resolve extension references, a Plan is what we use to construct that ExtensionSet.
+  // (It should be an optional output later.) In particular, it does not need to be kept
+  // alive nor does the serialized plan- none of the arrow:: objects in the output will
+  // contain references to memory owned by either.
+  auto maybe_decls = eng::DeserializePlan(*serialized_plan, consumer_factory);
+  ABORT_ON_FAILURE(maybe_decls.status());
+  std::vector<cp::Declaration> decls = std::move(maybe_decls).ValueOrDie();
+
+  // It's safe to drop the serialized plan; we don't leave references to its memory
+  serialized_plan.reset();
+
+  // Construct an empty plan (note: configure Function registry and ThreadPool here)
+  auto maybe_plan = cp::ExecPlan::Make();
+  ABORT_ON_FAILURE(maybe_plan.status());
+  std::shared_ptr<cp::ExecPlan> plan = std::move(maybe_plan).ValueOrDie();
+
+  for (const cp::Declaration& decl : decls) {
+    // Add decl to plan (note: configure ExecNode registry here)
+    ABORT_ON_FAILURE(decl.AddToPlan(plan.get()).status());
+  }
+
+  // Validate the plan and print it to stdout
+  ABORT_ON_FAILURE(plan->Validate());
+  std::cout << std::string('#', 50) << " produced arrow::ExecPlan:" << std::endl;
+  std::cout << plan->ToString() << std::endl;
+
+  // Start the plan...
+  std::cout << std::string('#', 50) << " consuming batches:" << std::endl;
+  ABORT_ON_FAILURE(plan->StartProducing());
+
+  // ... and wait for it to finish
+  ABORT_ON_FAILURE(plan->finished().status());
+  return EXIT_SUCCESS;
+}
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer() {
+  // Emulate server interaction by parsing hard coded JSON
+  return eng::internal::SubstraitFromJSON("Plan", R"({
+    "relations": [
+      {"rel": {
+        "read": {
+          "base_schema": {
+            "struct": {
+              "types": [ {"i64": {}}, {"bool": {}} ]
+            },
+            "names": ["i", "b"]
+          },
+          "filter": {
+            "selection": {
+              "directReference": {
+                "structField": {
+                  "field": 1
+                }
+              }
+            }
+          },
+          "local_files": {
+            "items": [
+              {
+                "uri_file": "file:///tmp/dat1.parquet",
+                "format": "FILE_FORMAT_PARQUET"
+              },
+              {
+                "uri_file": "file:///tmp/dat2.parquet",
+                "format": "FILE_FORMAT_PARQUET"
+              }
+            ]
+          }
+        }
+      }}
+    ],
+    "extension_uris": [
+      {
+        "extension_uri_anchor": 7,
+        "uri": "https://github.com/apache/arrow/blob/master/format/substrait/extension_types.yaml"
+      }
+    ],
+    "extensions": [
+      {"extension_type": {
+        "extension_uri_reference": 7,
+        "type_anchor": 42,
+        "name": "null"
+      }},
+      {"extension_type_variation": {
+        "extension_uri_reference": 7,
+        "type_variation_anchor": 23,
+        "name": "u8"
+      }},
+      {"extension_function": {
+        "extension_uri_reference": 7,
+        "function_anchor": 42,
+        "name": "add"
+      }}
+    ]

Review comment:
       For an example plan this is a little confusing as we never reference `null`, `u8`, or `add` in our plan yet it also doesn't feel like this is the complete set of potential extension types / functions.

##########
File path: cpp/src/arrow/array/builder_base.h
##########
@@ -286,13 +287,27 @@ ARROW_EXPORT
 Status MakeBuilder(MemoryPool* pool, const std::shared_ptr<DataType>& type,
                    std::unique_ptr<ArrayBuilder>* out);
 
+inline Result<std::unique_ptr<ArrayBuilder>> MakeBuilder(

Review comment:
       I don't think it's a problem by any means but I'm curious why the choice to inline here instead of using the .cc file.

##########
File path: cpp/src/arrow/datum.h
##########
@@ -153,6 +153,13 @@ struct ARROW_EXPORT Datum {
       : Datum(std::shared_ptr<typename std::conditional<IsArray, Array, Scalar>::type>(
             value)) {}
 
+  // Cast from subtypes of Array or Scalar to Datum
+  template <typename T, bool IsArray = std::is_base_of<Array, T>::value,
+            bool IsScalar = std::is_base_of<Scalar, T>::value,
+            typename = enable_if_t<IsArray || IsScalar>>
+  Datum(T&& value)  // NOLINT implicit conversion

Review comment:
       Very cool.  I needed this exact thing the other day when building a query plan by hand with literal scalars.

##########
File path: cpp/examples/arrow/engine_substrait_consumption.cc
##########
@@ -0,0 +1,187 @@
+// 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/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/compute/exec/options.h>
+#include <arrow/engine/substrait/serde.h>
+
+#include <cstdlib>
+#include <iostream>
+#include <memory>
+#include <vector>
+
+namespace eng = ::arrow::engine;
+namespace cp = ::arrow::compute;
+
+#define ABORT_ON_FAILURE(expr)                     \
+  do {                                             \
+    arrow::Status status_ = (expr);                \
+    if (!status_.ok()) {                           \
+      std::cerr << status_.message() << std::endl; \
+      abort();                                     \
+    }                                              \
+  } while (0);
+
+arrow::Future<std::shared_ptr<arrow::Buffer>> GetSubstraitFromServer();
+
+class IgnoringConsumer : public cp::SinkNodeConsumer {
+ public:
+  explicit IgnoringConsumer(size_t tag) : tag_{tag} {}
+
+  arrow::Status Consume(cp::ExecBatch batch) override {
+    // Consume a batch of data
+    // (just print its row count to stdout)
+    std::cout << "-" << tag_ << " consumed " << batch.length << " rows" << std::endl;
+    return arrow::Status::OK();
+  }
+
+  arrow::Future<> Finish() override {
+    // Signal to the consumer that the last batch has been delivered
+    // (we don't do any real work in this consumer so mark it finished immediately)
+    //
+    // The returned future should only finish when all outstanding tasks have completed
+    // (after this method is called Consume is guaranteed not to be called again)
+    std::cout << "-" << tag_ << " finished" << std::endl;
+    return arrow::Future<>::MakeFinished();
+  }
+
+ private:
+  size_t tag_;
+};
+
+int main(int argc, char** argv) {
+  // Plans arrive at the consumer serialized in a substrait-formatted Buffer
+  auto maybe_serialized_plan = GetSubstraitFromServer().result();
+  ABORT_ON_FAILURE(maybe_serialized_plan.status());
+  std::shared_ptr<arrow::Buffer> serialized_plan =
+      std::move(maybe_serialized_plan).ValueOrDie();
+
+  // Print the received plan to stdout as JSON
+  arrow::Result<std::string> maybe_plan_json =
+      eng::internal::SubstraitToJSON("Plan", *serialized_plan);
+  ABORT_ON_FAILURE(maybe_plan_json.status());
+  std::cout << std::string('#', 50) << " received substrait::Plan:" << std::endl;
+  std::cout << maybe_plan_json.ValueOrDie() << std::endl;
+
+  // Deserializing a plan requires a factory for consumers: each time a sink node is
+  // deserialized, a consumer is constructed into which its batches will be piped.
+  std::vector<std::shared_ptr<cp::SinkNodeConsumer>> consumers;
+  std::function<std::shared_ptr<cp::SinkNodeConsumer>()> consumer_factory = [&] {
+    // All batches produced by the plan will be fed into IgnoringConsumers:
+    auto tag = consumers.size();
+    consumers.emplace_back(new IgnoringConsumer{tag});
+    return consumers.back();
+  };
+
+  // NOTE Although most of the Deserialize functions require a const ExtensionSet& to
+  // resolve extension references, a Plan is what we use to construct that ExtensionSet.
+  // (It should be an optional output later.) In particular, it does not need to be kept
+  // alive nor does the serialized plan- none of the arrow:: objects in the output will
+  // contain references to memory owned by either.
+  auto maybe_decls = eng::DeserializePlan(*serialized_plan, consumer_factory);
+  ABORT_ON_FAILURE(maybe_decls.status());
+  std::vector<cp::Declaration> decls = std::move(maybe_decls).ValueOrDie();
+
+  // It's safe to drop the serialized plan; we don't leave references to its memory
+  serialized_plan.reset();
+
+  // Construct an empty plan (note: configure Function registry and ThreadPool here)
+  auto maybe_plan = cp::ExecPlan::Make();
+  ABORT_ON_FAILURE(maybe_plan.status());
+  std::shared_ptr<cp::ExecPlan> plan = std::move(maybe_plan).ValueOrDie();
+
+  for (const cp::Declaration& decl : decls) {
+    // Add decl to plan (note: configure ExecNode registry here)

Review comment:
       This seems like an odd place to configure the `ExecNode` registry.  I'll buy "the ExecNode factory must be registered before this point."  I would guess that most users (and perhaps this example) will be configuring the exec node registry before they ever go out and fetch the plan from the server.

##########
File path: cpp/src/arrow/engine/simple_extension_type_internal.h
##########
@@ -0,0 +1,183 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include "arrow/extension_type.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/reflection_internal.h"
+#include "arrow/util/string.h"
+
+namespace arrow {
+namespace engine {
+
+template <const util::string_view& kExtensionName, typename Params,
+          typename ParamsProperties, const ParamsProperties* kProperties,
+          std::shared_ptr<DataType> GetStorage(const Params&)>
+class SimpleExtensionType : public ExtensionType {
+ public:
+  using ParamsType = Params;
+
+  explicit SimpleExtensionType(std::shared_ptr<DataType> storage_type, Params params = {})
+      : ExtensionType(std::move(storage_type)), params_(std::move(params)) {}
+
+  static std::shared_ptr<DataType> Make(Params params) {
+    auto storage_type = GetStorage(params);
+    return std::make_shared<SimpleExtensionType>(std::move(storage_type),
+                                                 std::move(params));
+  }
+
+  static const Params* GetIf(const DataType& type) {
+    if (type.id() != Type::EXTENSION) return nullptr;
+
+    const auto& ext_type = ::arrow::internal::checked_cast<const ExtensionType&>(type);
+    if (ext_type.extension_name() != kExtensionName) return nullptr;
+
+    return &::arrow::internal::checked_cast<const SimpleExtensionType&>(type).params_;

Review comment:
       Would it be simpler to just `checked_cast` directly to `SimpleExtensionType`?

##########
File path: cpp/src/arrow/type.cc
##########
@@ -1544,7 +1544,7 @@ Result<std::shared_ptr<Schema>> Schema::AddField(
 Result<std::shared_ptr<Schema>> Schema::SetField(
     int i, const std::shared_ptr<Field>& field) const {
   if (i < 0 || i > this->num_fields()) {
-    return Status::Invalid("Invalid column index to set field.");
+    return Status::Invalid("Invalid column index to add field.");

Review comment:
       The method is named `SetField` and it does appear to be replacing an existing field.  Why is `add` a better choice here?

##########
File path: cpp/src/arrow/compute/exec/expression_internal.h
##########
@@ -29,9 +29,6 @@
 #include "arrow/util/logging.h"
 
 namespace arrow {
-
-using internal::checked_cast;

Review comment:
       Are we moving away from this pattern?

##########
File path: cpp/src/arrow/scalar.h
##########
@@ -345,8 +345,8 @@ struct ARROW_EXPORT TimestampScalar : public TemporalScalar<TimestampType> {
   using TemporalScalar<TimestampType>::TemporalScalar;
 
   TimestampScalar(typename TemporalScalar<TimestampType>::ValueType value,
-                  TimeUnit::type unit)
-      : TimestampScalar(std::move(value), timestamp(unit)) {}
+                  TimeUnit::type unit, std::string tz = "")

Review comment:
       I'm not sure it makes sense for `tz` to have a default value but I suppose there is some precedent for it on the python side.

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?
+#
+# Ultimately these declarations are a promise which needs to be backed by
+# equivalent serde in c++. For example, consider u8: when serializing to
+# substrait, we need to wrap instances of arrow::uint8 into the type
+# variation listed below. It would be ideal if we could SinglePointOfTruth
+# this correspondence; either generating c++ from the YAML or YAML from the
+# c++.
+#
+# At present (AFAICT) it's not valid to make this user extensible because
+# even if a user adds their custom scalar function to the registry *and*
+# defines the mapping from that scalar function to a substrait::ExtensionFunction
+# the corresponding YAML doesn't exist at any URI and so it can't be used in
+# substrait. Perhaps we could still help that case by providing a tool to
+# generate YAML from functions; that'd simplify the lives of people trying to
+# write arrow::compute::Functions to "define the function and if you want to
+# reference it from substrait generate this YAML and put it at some URI".
+#
+# In any case for the foreseeable future generation would be far too brittle;
+# URIs will not be accessed by anything but humans and the YAML is effectively
+# structured documentation. Thus extenders should pass the URI in the same way
+# they pass a description string; it's opaque to anything in arrow.
+#
+# We'll need a global mapping which contains:
+#  arrow::uint8 <-> {
+#   uri: "https://github.com/apache/arrow/blob//format/substrait/extension_types.yaml",
+#   name: "u8"
+#  }

Review comment:
       Does this PR not include that mapping?

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?
+#
+# Ultimately these declarations are a promise which needs to be backed by
+# equivalent serde in c++. For example, consider u8: when serializing to
+# substrait, we need to wrap instances of arrow::uint8 into the type
+# variation listed below. It would be ideal if we could SinglePointOfTruth
+# this correspondence; either generating c++ from the YAML or YAML from the
+# c++.
+#
+# At present (AFAICT) it's not valid to make this user extensible because
+# even if a user adds their custom scalar function to the registry *and*
+# defines the mapping from that scalar function to a substrait::ExtensionFunction
+# the corresponding YAML doesn't exist at any URI and so it can't be used in
+# substrait. Perhaps we could still help that case by providing a tool to
+# generate YAML from functions; that'd simplify the lives of people trying to
+# write arrow::compute::Functions to "define the function and if you want to
+# reference it from substrait generate this YAML and put it at some URI".
+#
+# In any case for the foreseeable future generation would be far too brittle;
+# URIs will not be accessed by anything but humans and the YAML is effectively
+# structured documentation. Thus extenders should pass the URI in the same way
+# they pass a description string; it's opaque to anything in arrow.

Review comment:
       I'm not sure you can say both "the corresponding YAML doesn't exist at any URI and so it can't be used in substrait" AND "URIs will not be accessed by anything but humans and the YAML is effectively structured documentation".
   
   The URI is basically a namespace.  XML does this with xmlns.  I've written plenty of XML documents with custom URI namespaces that never actually resolved to anything.  Also, a URI is a name, it doesn't have to resolve to anything.  It's perfectly valid to use a URI and never actually host the thing.

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?
+#
+# Ultimately these declarations are a promise which needs to be backed by
+# equivalent serde in c++. For example, consider u8: when serializing to
+# substrait, we need to wrap instances of arrow::uint8 into the type
+# variation listed below. It would be ideal if we could SinglePointOfTruth
+# this correspondence; either generating c++ from the YAML or YAML from the
+# c++.
+#
+# At present (AFAICT) it's not valid to make this user extensible because
+# even if a user adds their custom scalar function to the registry *and*
+# defines the mapping from that scalar function to a substrait::ExtensionFunction
+# the corresponding YAML doesn't exist at any URI and so it can't be used in
+# substrait. Perhaps we could still help that case by providing a tool to
+# generate YAML from functions; that'd simplify the lives of people trying to
+# write arrow::compute::Functions to "define the function and if you want to
+# reference it from substrait generate this YAML and put it at some URI".
+#
+# In any case for the foreseeable future generation would be far too brittle;
+# URIs will not be accessed by anything but humans and the YAML is effectively
+# structured documentation. Thus extenders should pass the URI in the same way
+# they pass a description string; it's opaque to anything in arrow.
+#
+# We'll need a global mapping which contains:
+#  arrow::uint8 <-> {
+#   uri: "https://github.com/apache/arrow/blob//format/substrait/extension_types.yaml",
+#   name: "u8"
+#  }
+#
+# And additionally convenient builders for subsets of that mapping during serde:
+#  arrow::uint8 <-> reference to anchor
+#
+# Worth noting: it's valid to have multiple ExtensionFunctions with the same name,
+# as long as all impls' mangled ("compound") names are unique.
+# Also worth noting: Options may only be enumerations. This will get strange with
+# functions like is_in, where we'll have to require that one argument is a literal
+# list or something.

Review comment:
       This feels like a comment that maybe belongs in a PR somewhere.

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?
+#
+# Ultimately these declarations are a promise which needs to be backed by
+# equivalent serde in c++. For example, consider u8: when serializing to
+# substrait, we need to wrap instances of arrow::uint8 into the type
+# variation listed below. It would be ideal if we could SinglePointOfTruth
+# this correspondence; either generating c++ from the YAML or YAML from the
+# c++.
+#
+# At present (AFAICT) it's not valid to make this user extensible because
+# even if a user adds their custom scalar function to the registry *and*
+# defines the mapping from that scalar function to a substrait::ExtensionFunction
+# the corresponding YAML doesn't exist at any URI and so it can't be used in
+# substrait. Perhaps we could still help that case by providing a tool to
+# generate YAML from functions; that'd simplify the lives of people trying to
+# write arrow::compute::Functions to "define the function and if you want to
+# reference it from substrait generate this YAML and put it at some URI".
+#
+# In any case for the foreseeable future generation would be far too brittle;
+# URIs will not be accessed by anything but humans and the YAML is effectively
+# structured documentation. Thus extenders should pass the URI in the same way
+# they pass a description string; it's opaque to anything in arrow.
+#
+# We'll need a global mapping which contains:
+#  arrow::uint8 <-> {
+#   uri: "https://github.com/apache/arrow/blob//format/substrait/extension_types.yaml",
+#   name: "u8"
+#  }
+#
+# And additionally convenient builders for subsets of that mapping during serde:
+#  arrow::uint8 <-> reference to anchor

Review comment:
       I don't understand this point.

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?
+#
+# Ultimately these declarations are a promise which needs to be backed by
+# equivalent serde in c++. For example, consider u8: when serializing to
+# substrait, we need to wrap instances of arrow::uint8 into the type
+# variation listed below. It would be ideal if we could SinglePointOfTruth
+# this correspondence; either generating c++ from the YAML or YAML from the
+# c++.
+#
+# At present (AFAICT) it's not valid to make this user extensible because
+# even if a user adds their custom scalar function to the registry *and*
+# defines the mapping from that scalar function to a substrait::ExtensionFunction
+# the corresponding YAML doesn't exist at any URI and so it can't be used in
+# substrait. Perhaps we could still help that case by providing a tool to
+# generate YAML from functions; that'd simplify the lives of people trying to
+# write arrow::compute::Functions to "define the function and if you want to
+# reference it from substrait generate this YAML and put it at some URI".
+#
+# In any case for the foreseeable future generation would be far too brittle;
+# URIs will not be accessed by anything but humans and the YAML is effectively
+# structured documentation. Thus extenders should pass the URI in the same way
+# they pass a description string; it's opaque to anything in arrow.
+#
+# We'll need a global mapping which contains:
+#  arrow::uint8 <-> {
+#   uri: "https://github.com/apache/arrow/blob//format/substrait/extension_types.yaml",
+#   name: "u8"
+#  }
+#
+# And additionally convenient builders for subsets of that mapping during serde:
+#  arrow::uint8 <-> reference to anchor
+#
+# Worth noting: it's valid to have multiple ExtensionFunctions with the same name,
+# as long as all impls' mangled ("compound") names are unique.
+# Also worth noting: Options may only be enumerations. This will get strange with
+# functions like is_in, where we'll have to require that one argument is a literal
+# list or something.
+
+type_variations:
+  - parent: i8
+    name: u8

Review comment:
       I'm not sure if an unsigned integer is a "type variation" or a separate type.

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?
+#
+# Ultimately these declarations are a promise which needs to be backed by
+# equivalent serde in c++. For example, consider u8: when serializing to
+# substrait, we need to wrap instances of arrow::uint8 into the type
+# variation listed below. It would be ideal if we could SinglePointOfTruth
+# this correspondence; either generating c++ from the YAML or YAML from the
+# c++.

Review comment:
       Rather than talk about this in user facing documentation can we create a JIRA to investigate a single source of truth for Substrait YAML?

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?

Review comment:
       Let's create a JIRA to get to figure out the dictionary encoding.  At one point Substrait was considering a separate "encoding" or "specialization" property for types.  I think we backed out of that and decided those would just be custom types.  But I don't really remember where we landed.
   
   For timezone strings, I don't think we will ever need to bother with this.  In my mind, the timezone string should always be a function option and not a part of the data.  We can hash that out in a separate JIRA (i.e. separate from this PR and separate from a JIRA on figuring out dictionary encoding).

##########
File path: format/substrait/extension_types.yaml
##########
@@ -0,0 +1,121 @@
+# 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.
+
+# substrait::{ExtensionTypeVariation, ExtensionType}s
+# for wrapping types which appear in the arrow type system but
+# are not first-class in substrait. These include:
+# - null
+# - unsigned integers
+# - half-precision floating point numbers
+# - 32 bit times and dates
+# - timestamps with units other than microseconds
+# - timestamps with timezones other than UTC
+# - 256 bit decimals
+# - sparse and dense unions
+# - dictionary encoded types
+# - durations
+# - string and binary with 64 bit offsets
+# - list with 64 bit offsets
+# - interval<months: i32>
+# - interval<days: i32, millis: i32>
+# - interval<months: i32, days: i32, nanos: i64>
+# - arrow::ExtensionTypes
+
+# FIXME these extension types are not parameterizable, which means among
+# other things that we can't declare dictionary type here at all since
+# we'd have to declare a different dictionary type for all encoded types
+# (but that is an infinite space). Similarly, do we need to declare a
+# timestamp variation for all possible timezone strings?
+#
+# Ultimately these declarations are a promise which needs to be backed by
+# equivalent serde in c++. For example, consider u8: when serializing to
+# substrait, we need to wrap instances of arrow::uint8 into the type
+# variation listed below. It would be ideal if we could SinglePointOfTruth
+# this correspondence; either generating c++ from the YAML or YAML from the
+# c++.
+#
+# At present (AFAICT) it's not valid to make this user extensible because
+# even if a user adds their custom scalar function to the registry *and*
+# defines the mapping from that scalar function to a substrait::ExtensionFunction
+# the corresponding YAML doesn't exist at any URI and so it can't be used in
+# substrait. Perhaps we could still help that case by providing a tool to
+# generate YAML from functions; that'd simplify the lives of people trying to
+# write arrow::compute::Functions to "define the function and if you want to
+# reference it from substrait generate this YAML and put it at some URI".
+#
+# In any case for the foreseeable future generation would be far too brittle;
+# URIs will not be accessed by anything but humans and the YAML is effectively
+# structured documentation. Thus extenders should pass the URI in the same way
+# they pass a description string; it's opaque to anything in arrow.
+#
+# We'll need a global mapping which contains:
+#  arrow::uint8 <-> {
+#   uri: "https://github.com/apache/arrow/blob//format/substrait/extension_types.yaml",
+#   name: "u8"
+#  }
+#
+# And additionally convenient builders for subsets of that mapping during serde:
+#  arrow::uint8 <-> reference to anchor
+#
+# Worth noting: it's valid to have multiple ExtensionFunctions with the same name,
+# as long as all impls' mangled ("compound") names are unique.
+# Also worth noting: Options may only be enumerations. This will get strange with
+# functions like is_in, where we'll have to require that one argument is a literal
+# list or something.
+
+type_variations:
+  - parent: i8
+    name: u8
+    description: an unsigned 8 bit integer
+    functions: SEPARATE
+  - parent: i16
+    name: u16
+    description: an unsigned 16 bit integer
+    functions: SEPARATE
+  - parent: i32
+    name: u32
+    description: an unsigned 32 bit integer
+    functions: SEPARATE
+  - parent: i32
+    name: u32
+    description: an unsigned 32 bit integer
+    functions: SEPARATE
+  - parent: i64
+    name: u64
+    description: an unsigned 64 bit integer
+    functions: SEPARATE
+
+  - parent: i16

Review comment:
       Is `i16` really the parent?  My understanding of "parent" is that it is a "semantic" parent although I don't get the point of specifying a parent at all if using `functions: SEPARATE` so... :shrug: 

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));
+
+    case substrait::Expression::Literal::kDate:
+      return Datum(Date64Scalar(static_cast<int64_t>(lit.date())));
+    case substrait::Expression::Literal::kTime:
+      return Datum(Time64Scalar(static_cast<int64_t>(lit.time()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kIntervalYearToMonth:
+    case substrait::Expression::Literal::kIntervalDayToSecond: {
+      Int32Builder builder;
+      std::shared_ptr<DataType> type;
+      if (lit.has_interval_year_to_month()) {
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().years()));
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().months()));
+        type = interval_year();
+      } else {
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().days()));
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().seconds()));
+        type = interval_day();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto array, builder.Finish());
+      return Datum(
+          ExtensionScalar(FixedSizeListScalar(std::move(array)), std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kUuid:
+      return Datum(ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.uuid()), uuid()));
+
+    case substrait::Expression::Literal::kFixedChar:
+      return Datum(
+          ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.fixed_char()),
+                          fixed_char(static_cast<int32_t>(lit.fixed_char().size()))));
+
+    case substrait::Expression::Literal::kVarChar:
+      return Datum(
+          ExtensionScalar(StringScalar(lit.var_char().value()),
+                          varchar(static_cast<int32_t>(lit.var_char().length()))));
+
+    case substrait::Expression::Literal::kFixedBinary:
+      return Datum(FixedSizeBinaryScalarFromBytes(lit.fixed_binary()));
+
+    case substrait::Expression::Literal::kDecimal: {
+      if (lit.decimal().value().size() != sizeof(Decimal128)) {
+        return Status::Invalid("Decimal literal had ", lit.decimal().value().size(),
+                               " bytes (expected ", sizeof(Decimal128), ")");
+      }
+
+      Decimal128 value;
+      std::memcpy(value.mutable_native_endian_bytes(), lit.decimal().value().data(),
+                  sizeof(Decimal128));
+#if !ARROW_LITTLE_ENDIAN
+      std::reverse(value.mutable_native_endian_bytes(),
+                   value.mutable_native_endian_bytes() + sizeof(Decimal128));
+#endif
+      auto type = decimal128(lit.decimal().precision(), lit.decimal().scale());
+      return Datum(Decimal128Scalar(value, std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kStruct: {
+      const auto& struct_ = lit.struct_();
+
+      ScalarVector fields(struct_.fields_size());
+      std::vector<std::string> field_names(fields.size(), "");
+      for (int i = 0; i < struct_.fields_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(auto field, FromProto(struct_.fields(i), ext_set));
+        DCHECK(field.is_scalar());
+        fields[i] = field.scalar();
+      }
+      ARROW_ASSIGN_OR_RAISE(
+          auto scalar, StructScalar::Make(std::move(fields), std::move(field_names)));
+      return Datum(std::move(scalar));

Review comment:
       This matches the templated "Cast from subtypes of Array or Scalar to Datum" constructor that takes `const std::shared_ptr<T>&`

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }

Review comment:
       What does round tripping mean in this context?  We don't have any way of going back to a Substrait plan and, even if we did, I think that would be more for debugging purposes than anything else.
   
   It seems to me that Substrait is giving us more information than we need so we should just ignore it either way.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {

Review comment:
       ```suggestion
         for (const auto& if_ : if_then.ifs()) {
   ```

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"

Review comment:
       ```suggestion
   ```

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));
+
+    case substrait::Expression::Literal::kDate:
+      return Datum(Date64Scalar(static_cast<int64_t>(lit.date())));
+    case substrait::Expression::Literal::kTime:
+      return Datum(Time64Scalar(static_cast<int64_t>(lit.time()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kIntervalYearToMonth:
+    case substrait::Expression::Literal::kIntervalDayToSecond: {
+      Int32Builder builder;
+      std::shared_ptr<DataType> type;
+      if (lit.has_interval_year_to_month()) {
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().years()));
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().months()));
+        type = interval_year();
+      } else {
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().days()));
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().seconds()));
+        type = interval_day();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto array, builder.Finish());
+      return Datum(
+          ExtensionScalar(FixedSizeListScalar(std::move(array)), std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kUuid:
+      return Datum(ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.uuid()), uuid()));
+
+    case substrait::Expression::Literal::kFixedChar:
+      return Datum(
+          ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.fixed_char()),
+                          fixed_char(static_cast<int32_t>(lit.fixed_char().size()))));
+
+    case substrait::Expression::Literal::kVarChar:
+      return Datum(
+          ExtensionScalar(StringScalar(lit.var_char().value()),
+                          varchar(static_cast<int32_t>(lit.var_char().length()))));
+
+    case substrait::Expression::Literal::kFixedBinary:
+      return Datum(FixedSizeBinaryScalarFromBytes(lit.fixed_binary()));
+
+    case substrait::Expression::Literal::kDecimal: {
+      if (lit.decimal().value().size() != sizeof(Decimal128)) {
+        return Status::Invalid("Decimal literal had ", lit.decimal().value().size(),
+                               " bytes (expected ", sizeof(Decimal128), ")");
+      }
+
+      Decimal128 value;
+      std::memcpy(value.mutable_native_endian_bytes(), lit.decimal().value().data(),
+                  sizeof(Decimal128));
+#if !ARROW_LITTLE_ENDIAN
+      std::reverse(value.mutable_native_endian_bytes(),
+                   value.mutable_native_endian_bytes() + sizeof(Decimal128));
+#endif
+      auto type = decimal128(lit.decimal().precision(), lit.decimal().scale());
+      return Datum(Decimal128Scalar(value, std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kStruct: {
+      const auto& struct_ = lit.struct_();
+
+      ScalarVector fields(struct_.fields_size());
+      std::vector<std::string> field_names(fields.size(), "");
+      for (int i = 0; i < struct_.fields_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(auto field, FromProto(struct_.fields(i), ext_set));
+        DCHECK(field.is_scalar());
+        fields[i] = field.scalar();
+      }
+      ARROW_ASSIGN_OR_RAISE(
+          auto scalar, StructScalar::Make(std::move(fields), std::move(field_names)));
+      return Datum(std::move(scalar));
+    }
+
+    case substrait::Expression::Literal::kList: {
+      const auto& list = lit.list();
+      if (list.values_size() == 0) {
+        return Status::Invalid(
+            "substrait::Expression::Literal::List had no values; should have been an "
+            "substrait::Expression::Literal::EmptyList");
+      }
+
+      std::shared_ptr<DataType> element_type;
+
+      ScalarVector values(list.values_size());
+      for (int i = 0; i < list.values_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(auto value, FromProto(list.values(i), ext_set));
+        DCHECK(value.is_scalar());
+        values[i] = value.scalar();
+        if (element_type) {
+          if (!value.type()->Equals(*element_type)) {
+            return Status::Invalid(
+                list.DebugString(),
+                " has a value whose type doesn't match the other list values");
+          }
+        } else {
+          element_type = value.type();
+        }
+      }
+
+      ARROW_ASSIGN_OR_RAISE(auto builder, MakeBuilder(std::move(element_type)));

Review comment:
       ```suggestion
         ARROW_ASSIGN_OR_RAISE(auto builder, MakeBuilder(element_type));
   ```

##########
File path: cpp/src/arrow/datum.h
##########
@@ -153,6 +153,13 @@ struct ARROW_EXPORT Datum {
       : Datum(std::shared_ptr<typename std::conditional<IsArray, Array, Scalar>::type>(

Review comment:
       Since we're nearby can you take a look at this?  Why are we using `const std::shared_ptr<T>&`?  Is it inevitable we must make a copy of the `shared_ptr`?

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));
+
+    case substrait::Expression::Literal::kDate:
+      return Datum(Date64Scalar(static_cast<int64_t>(lit.date())));
+    case substrait::Expression::Literal::kTime:
+      return Datum(Time64Scalar(static_cast<int64_t>(lit.time()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kIntervalYearToMonth:
+    case substrait::Expression::Literal::kIntervalDayToSecond: {

Review comment:
       Arrow has interval types.  Why did we need extension types here?  `kIntervalYearToMonth` should map to https://github.com/apache/arrow/blob/40ac81087a9d6d97dae4b11c28a147176d22dc54/format/Schema.fbs#L361 and `kIntervalDayToSecond` should map to https://github.com/apache/arrow/blob/40ac81087a9d6d97dae4b11c28a147176d22dc54/format/Schema.fbs#L366

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));
+
+    case substrait::Expression::Literal::kDate:
+      return Datum(Date64Scalar(static_cast<int64_t>(lit.date())));

Review comment:
       I think a Substrait [Date](https://substrait.io/types/simple_logical_types/) maps to `Date<DAY>` which would be `Date32Scalar`.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));

Review comment:
       Minor nit: for consistency's sake we could add a constructor:
   
   ```
   BinaryScalar::BinaryScalar(const std::string&);
   ```
   

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));
+
+    case substrait::Expression::Literal::kDate:
+      return Datum(Date64Scalar(static_cast<int64_t>(lit.date())));
+    case substrait::Expression::Literal::kTime:
+      return Datum(Time64Scalar(static_cast<int64_t>(lit.time()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kIntervalYearToMonth:
+    case substrait::Expression::Literal::kIntervalDayToSecond: {
+      Int32Builder builder;
+      std::shared_ptr<DataType> type;
+      if (lit.has_interval_year_to_month()) {
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().years()));
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().months()));
+        type = interval_year();
+      } else {
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().days()));
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().seconds()));
+        type = interval_day();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto array, builder.Finish());
+      return Datum(
+          ExtensionScalar(FixedSizeListScalar(std::move(array)), std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kUuid:
+      return Datum(ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.uuid()), uuid()));
+
+    case substrait::Expression::Literal::kFixedChar:
+      return Datum(
+          ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.fixed_char()),
+                          fixed_char(static_cast<int32_t>(lit.fixed_char().size()))));
+
+    case substrait::Expression::Literal::kVarChar:
+      return Datum(
+          ExtensionScalar(StringScalar(lit.var_char().value()),
+                          varchar(static_cast<int32_t>(lit.var_char().length()))));
+
+    case substrait::Expression::Literal::kFixedBinary:
+      return Datum(FixedSizeBinaryScalarFromBytes(lit.fixed_binary()));
+
+    case substrait::Expression::Literal::kDecimal: {
+      if (lit.decimal().value().size() != sizeof(Decimal128)) {
+        return Status::Invalid("Decimal literal had ", lit.decimal().value().size(),
+                               " bytes (expected ", sizeof(Decimal128), ")");
+      }
+
+      Decimal128 value;
+      std::memcpy(value.mutable_native_endian_bytes(), lit.decimal().value().data(),
+                  sizeof(Decimal128));
+#if !ARROW_LITTLE_ENDIAN
+      std::reverse(value.mutable_native_endian_bytes(),
+                   value.mutable_native_endian_bytes() + sizeof(Decimal128));
+#endif
+      auto type = decimal128(lit.decimal().precision(), lit.decimal().scale());
+      return Datum(Decimal128Scalar(value, std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kStruct: {
+      const auto& struct_ = lit.struct_();
+
+      ScalarVector fields(struct_.fields_size());
+      std::vector<std::string> field_names(fields.size(), "");

Review comment:
       Maybe worth a comment here pointing out that, in Substrait, a struct literal has no field names.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));
+
+    case substrait::Expression::Literal::kDate:
+      return Datum(Date64Scalar(static_cast<int64_t>(lit.date())));
+    case substrait::Expression::Literal::kTime:
+      return Datum(Time64Scalar(static_cast<int64_t>(lit.time()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kIntervalYearToMonth:
+    case substrait::Expression::Literal::kIntervalDayToSecond: {
+      Int32Builder builder;
+      std::shared_ptr<DataType> type;
+      if (lit.has_interval_year_to_month()) {
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().years()));
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().months()));
+        type = interval_year();
+      } else {
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().days()));
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().seconds()));
+        type = interval_day();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto array, builder.Finish());
+      return Datum(
+          ExtensionScalar(FixedSizeListScalar(std::move(array)), std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kUuid:
+      return Datum(ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.uuid()), uuid()));
+
+    case substrait::Expression::Literal::kFixedChar:
+      return Datum(
+          ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.fixed_char()),
+                          fixed_char(static_cast<int32_t>(lit.fixed_char().size()))));
+
+    case substrait::Expression::Literal::kVarChar:
+      return Datum(
+          ExtensionScalar(StringScalar(lit.var_char().value()),
+                          varchar(static_cast<int32_t>(lit.var_char().length()))));

Review comment:
       Technically `kVarChar` is not part of the Substrait spec.  But maybe that's just a "not part of it yet" kind of thing and so forward planning isn't a bad idea.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));
+
+    case substrait::Expression::Literal::kDate:
+      return Datum(Date64Scalar(static_cast<int64_t>(lit.date())));
+    case substrait::Expression::Literal::kTime:
+      return Datum(Time64Scalar(static_cast<int64_t>(lit.time()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kIntervalYearToMonth:
+    case substrait::Expression::Literal::kIntervalDayToSecond: {
+      Int32Builder builder;
+      std::shared_ptr<DataType> type;
+      if (lit.has_interval_year_to_month()) {
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().years()));
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().months()));
+        type = interval_year();
+      } else {
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().days()));
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().seconds()));
+        type = interval_day();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto array, builder.Finish());
+      return Datum(
+          ExtensionScalar(FixedSizeListScalar(std::move(array)), std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kUuid:
+      return Datum(ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.uuid()), uuid()));
+
+    case substrait::Expression::Literal::kFixedChar:
+      return Datum(
+          ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.fixed_char()),
+                          fixed_char(static_cast<int32_t>(lit.fixed_char().size()))));
+
+    case substrait::Expression::Literal::kVarChar:
+      return Datum(
+          ExtensionScalar(StringScalar(lit.var_char().value()),
+                          varchar(static_cast<int32_t>(lit.var_char().length()))));
+
+    case substrait::Expression::Literal::kFixedBinary:
+      return Datum(FixedSizeBinaryScalarFromBytes(lit.fixed_binary()));
+
+    case substrait::Expression::Literal::kDecimal: {
+      if (lit.decimal().value().size() != sizeof(Decimal128)) {
+        return Status::Invalid("Decimal literal had ", lit.decimal().value().size(),
+                               " bytes (expected ", sizeof(Decimal128), ")");
+      }
+
+      Decimal128 value;
+      std::memcpy(value.mutable_native_endian_bytes(), lit.decimal().value().data(),
+                  sizeof(Decimal128));
+#if !ARROW_LITTLE_ENDIAN
+      std::reverse(value.mutable_native_endian_bytes(),
+                   value.mutable_native_endian_bytes() + sizeof(Decimal128));
+#endif
+      auto type = decimal128(lit.decimal().precision(), lit.decimal().scale());
+      return Datum(Decimal128Scalar(value, std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kStruct: {
+      const auto& struct_ = lit.struct_();
+
+      ScalarVector fields(struct_.fields_size());
+      std::vector<std::string> field_names(fields.size(), "");
+      for (int i = 0; i < struct_.fields_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(auto field, FromProto(struct_.fields(i), ext_set));
+        DCHECK(field.is_scalar());
+        fields[i] = field.scalar();
+      }
+      ARROW_ASSIGN_OR_RAISE(
+          auto scalar, StructScalar::Make(std::move(fields), std::move(field_names)));
+      return Datum(std::move(scalar));
+    }
+
+    case substrait::Expression::Literal::kList: {
+      const auto& list = lit.list();
+      if (list.values_size() == 0) {
+        return Status::Invalid(
+            "substrait::Expression::Literal::List had no values; should have been an "
+            "substrait::Expression::Literal::EmptyList");
+      }
+
+      std::shared_ptr<DataType> element_type;
+
+      ScalarVector values(list.values_size());
+      for (int i = 0; i < list.values_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(auto value, FromProto(list.values(i), ext_set));
+        DCHECK(value.is_scalar());
+        values[i] = value.scalar();
+        if (element_type) {
+          if (!value.type()->Equals(*element_type)) {
+            return Status::Invalid(
+                list.DebugString(),
+                " has a value whose type doesn't match the other list values");
+          }
+        } else {
+          element_type = value.type();
+        }
+      }
+
+      ARROW_ASSIGN_OR_RAISE(auto builder, MakeBuilder(std::move(element_type)));
+      RETURN_NOT_OK(builder->AppendScalars(values));
+      ARROW_ASSIGN_OR_RAISE(auto arr, builder->Finish());
+      return Datum(ListScalar(std::move(arr)));
+    }
+
+    case substrait::Expression::Literal::kMap: {
+      const auto& map = lit.map();
+      if (map.key_values_size() == 0) {
+        return Status::Invalid(
+            "substrait::Expression::Literal::Map had no values; should have been an "
+            "substrait::Expression::Literal::EmptyMap");
+      }
+
+      std::shared_ptr<DataType> key_type, value_type;
+      ScalarVector keys(map.key_values_size()), values(map.key_values_size());
+      for (int i = 0; i < map.key_values_size(); ++i) {
+        const auto& kv = map.key_values(i);
+
+        static const std::array<char const*, 4> kMissing = {"key and value", "value",
+                                                            "key", nullptr};
+        if (auto missing = kMissing[kv.has_key() + kv.has_value() * 2]) {
+          return Status::Invalid("While converting to MapScalar encountered missing ",
+                                 missing, " in ", map.DebugString());
+        }
+        ARROW_ASSIGN_OR_RAISE(auto key, FromProto(kv.key(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto value, FromProto(kv.value(), ext_set));
+
+        DCHECK(key.is_scalar());
+        DCHECK(value.is_scalar());
+
+        keys[i] = key.scalar();
+        values[i] = value.scalar();
+
+        if (key_type) {
+          if (!key.type()->Equals(*key_type)) {
+            return Status::Invalid(map.DebugString(),
+                                   " has a key whose type doesn't match key_type");
+          }
+        } else {
+          key_type = value.type();
+        }
+
+        if (value_type) {
+          if (!value.type()->Equals(*value_type)) {
+            return Status::Invalid(map.DebugString(),
+                                   " has a value whose type doesn't match value_type");
+          }
+        } else {
+          value_type = value.type();
+        }
+      }
+
+      ARROW_ASSIGN_OR_RAISE(auto key_builder, MakeBuilder(key_type));
+      ARROW_ASSIGN_OR_RAISE(auto value_builder, MakeBuilder(value_type));
+      RETURN_NOT_OK(key_builder->AppendScalars(keys));
+      RETURN_NOT_OK(value_builder->AppendScalars(values));
+      ARROW_ASSIGN_OR_RAISE(auto key_arr, key_builder->Finish());
+      ARROW_ASSIGN_OR_RAISE(auto value_arr, value_builder->Finish());
+      ARROW_ASSIGN_OR_RAISE(
+          auto kv_arr,
+          StructArray::Make(ArrayVector{std::move(key_arr), std::move(value_arr)},
+                            std::vector<std::string>{"key", "value"}));
+      return Datum(std::make_shared<MapScalar>(std::move(kv_arr)));
+    }
+
+    case substrait::Expression::Literal::kEmptyList: {
+      ARROW_ASSIGN_OR_RAISE(auto type_nullable,
+                            FromProto(lit.empty_list().type(), ext_set));
+      ARROW_ASSIGN_OR_RAISE(auto values, MakeEmptyArray(type_nullable.first));
+      return ListScalar{std::move(values)};
+    }
+
+    case substrait::Expression::Literal::kEmptyMap: {
+      ARROW_ASSIGN_OR_RAISE(auto key_type_nullable,
+                            FromProto(lit.empty_map().key(), ext_set));
+      ARROW_ASSIGN_OR_RAISE(auto keys,
+                            MakeEmptyArray(std::move(key_type_nullable.first)));
+
+      ARROW_ASSIGN_OR_RAISE(auto value_type_nullable,
+                            FromProto(lit.empty_map().value(), ext_set));
+      ARROW_ASSIGN_OR_RAISE(auto values,
+                            MakeEmptyArray(std::move(value_type_nullable.first)));
+
+      auto map_type = std::make_shared<MapType>(keys->type(), values->type());
+      ARROW_ASSIGN_OR_RAISE(
+          auto key_values,
+          StructArray::Make(
+              {std::move(keys), std::move(values)},
+              checked_cast<const ListType&>(*map_type).value_type()->fields()));
+
+      return MapScalar{std::move(key_values)};
+    }
+
+    case substrait::Expression::Literal::kNull: {
+      ARROW_ASSIGN_OR_RAISE(auto type_nullable, FromProto(lit.null(), ext_set));
+      if (!type_nullable.second) {
+        return Status::Invalid("Null literal ", lit.DebugString(),
+                               " is of non-nullable type");
+      }
+
+      return Datum(MakeNullScalar(std::move(type_nullable.first)));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::Datum from ", lit.DebugString());

Review comment:
       ```suggestion
     return Status::NotImplemented("conversion to arrow::Datum from Substrait literal ", lit.DebugString());
   ```
   I don't know if `lit.DebugString()` would obviously be a substrait literal or not (if it is obvious in some way then this change isn't needed).  I'm imagining a user that tries to run some Ibis query and gets this message and submits it to JIRA.
   
   The same logic applies to the other fallback `NotImplemented` cases.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));

Review comment:
       Nit: `TimestampTzTimezoneString` is a weird name for "UTC" but I can kind of see the reasoning.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",
+                                expr.DebugString());
+}
+
+Result<Datum> FromProto(const substrait::Expression::Literal& lit,
+                        const ExtensionSet& ext_set) {
+  if (lit.nullable()) {
+    // FIXME not sure how this field should be interpreted and there's no way to round
+    // trip it through arrow
+    return Status::Invalid(
+        "Nullable Literals - Literal.nullable must be left at the default");
+  }
+
+  switch (lit.literal_type_case()) {
+    case substrait::Expression::Literal::kBoolean:
+      return Datum(lit.boolean());
+
+    case substrait::Expression::Literal::kI8:
+      return Datum(static_cast<int8_t>(lit.i8()));
+    case substrait::Expression::Literal::kI16:
+      return Datum(static_cast<int16_t>(lit.i16()));
+    case substrait::Expression::Literal::kI32:
+      return Datum(static_cast<int32_t>(lit.i32()));
+    case substrait::Expression::Literal::kI64:
+      return Datum(static_cast<int64_t>(lit.i64()));
+
+    case substrait::Expression::Literal::kFp32:
+      return Datum(lit.fp32());
+    case substrait::Expression::Literal::kFp64:
+      return Datum(lit.fp64());
+
+    case substrait::Expression::Literal::kString:
+      return Datum(lit.string());
+    case substrait::Expression::Literal::kBinary:
+      return Datum(BinaryScalar(Buffer::FromString(lit.binary())));
+
+    case substrait::Expression::Literal::kTimestamp:
+      return Datum(
+          TimestampScalar(static_cast<int64_t>(lit.timestamp()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kTimestampTz:
+      return Datum(TimestampScalar(static_cast<int64_t>(lit.timestamp_tz()),
+                                   TimeUnit::MICRO, TimestampTzTimezoneString()));
+
+    case substrait::Expression::Literal::kDate:
+      return Datum(Date64Scalar(static_cast<int64_t>(lit.date())));
+    case substrait::Expression::Literal::kTime:
+      return Datum(Time64Scalar(static_cast<int64_t>(lit.time()), TimeUnit::MICRO));
+
+    case substrait::Expression::Literal::kIntervalYearToMonth:
+    case substrait::Expression::Literal::kIntervalDayToSecond: {
+      Int32Builder builder;
+      std::shared_ptr<DataType> type;
+      if (lit.has_interval_year_to_month()) {
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().years()));
+        RETURN_NOT_OK(builder.Append(lit.interval_year_to_month().months()));
+        type = interval_year();
+      } else {
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().days()));
+        RETURN_NOT_OK(builder.Append(lit.interval_day_to_second().seconds()));
+        type = interval_day();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto array, builder.Finish());
+      return Datum(
+          ExtensionScalar(FixedSizeListScalar(std::move(array)), std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kUuid:
+      return Datum(ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.uuid()), uuid()));
+
+    case substrait::Expression::Literal::kFixedChar:
+      return Datum(
+          ExtensionScalar(FixedSizeBinaryScalarFromBytes(lit.fixed_char()),
+                          fixed_char(static_cast<int32_t>(lit.fixed_char().size()))));
+
+    case substrait::Expression::Literal::kVarChar:
+      return Datum(
+          ExtensionScalar(StringScalar(lit.var_char().value()),
+                          varchar(static_cast<int32_t>(lit.var_char().length()))));
+
+    case substrait::Expression::Literal::kFixedBinary:
+      return Datum(FixedSizeBinaryScalarFromBytes(lit.fixed_binary()));
+
+    case substrait::Expression::Literal::kDecimal: {
+      if (lit.decimal().value().size() != sizeof(Decimal128)) {
+        return Status::Invalid("Decimal literal had ", lit.decimal().value().size(),
+                               " bytes (expected ", sizeof(Decimal128), ")");
+      }
+
+      Decimal128 value;
+      std::memcpy(value.mutable_native_endian_bytes(), lit.decimal().value().data(),
+                  sizeof(Decimal128));
+#if !ARROW_LITTLE_ENDIAN
+      std::reverse(value.mutable_native_endian_bytes(),
+                   value.mutable_native_endian_bytes() + sizeof(Decimal128));
+#endif
+      auto type = decimal128(lit.decimal().precision(), lit.decimal().scale());
+      return Datum(Decimal128Scalar(value, std::move(type)));
+    }
+
+    case substrait::Expression::Literal::kStruct: {
+      const auto& struct_ = lit.struct_();
+
+      ScalarVector fields(struct_.fields_size());
+      std::vector<std::string> field_names(fields.size(), "");
+      for (int i = 0; i < struct_.fields_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(auto field, FromProto(struct_.fields(i), ext_set));
+        DCHECK(field.is_scalar());
+        fields[i] = field.scalar();
+      }
+      ARROW_ASSIGN_OR_RAISE(
+          auto scalar, StructScalar::Make(std::move(fields), std::move(field_names)));
+      return Datum(std::move(scalar));
+    }
+
+    case substrait::Expression::Literal::kList: {
+      const auto& list = lit.list();
+      if (list.values_size() == 0) {
+        return Status::Invalid(
+            "substrait::Expression::Literal::List had no values; should have been an "
+            "substrait::Expression::Literal::EmptyList");
+      }
+
+      std::shared_ptr<DataType> element_type;
+
+      ScalarVector values(list.values_size());
+      for (int i = 0; i < list.values_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(auto value, FromProto(list.values(i), ext_set));
+        DCHECK(value.is_scalar());
+        values[i] = value.scalar();
+        if (element_type) {
+          if (!value.type()->Equals(*element_type)) {
+            return Status::Invalid(
+                list.DebugString(),
+                " has a value whose type doesn't match the other list values");
+          }
+        } else {
+          element_type = value.type();
+        }
+      }
+
+      ARROW_ASSIGN_OR_RAISE(auto builder, MakeBuilder(std::move(element_type)));
+      RETURN_NOT_OK(builder->AppendScalars(values));
+      ARROW_ASSIGN_OR_RAISE(auto arr, builder->Finish());
+      return Datum(ListScalar(std::move(arr)));
+    }
+
+    case substrait::Expression::Literal::kMap: {
+      const auto& map = lit.map();
+      if (map.key_values_size() == 0) {
+        return Status::Invalid(
+            "substrait::Expression::Literal::Map had no values; should have been an "
+            "substrait::Expression::Literal::EmptyMap");
+      }
+
+      std::shared_ptr<DataType> key_type, value_type;
+      ScalarVector keys(map.key_values_size()), values(map.key_values_size());
+      for (int i = 0; i < map.key_values_size(); ++i) {
+        const auto& kv = map.key_values(i);
+
+        static const std::array<char const*, 4> kMissing = {"key and value", "value",
+                                                            "key", nullptr};
+        if (auto missing = kMissing[kv.has_key() + kv.has_value() * 2]) {

Review comment:
       This is right on the edge of too clever for your own good :laughing: 

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;

Review comment:
       Will the error be clear enough here?  I suppose we can always figure it out but I wonder if it might be clearer to say that "we have only implemented direct references" or something to that effect.

##########
File path: cpp/src/arrow/engine/substrait/serde.h
##########
@@ -0,0 +1,79 @@
+// 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 <string>
+#include <vector>
+
+#include "arrow/buffer.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/engine/substrait/extension_set.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+
+using ConsumerFactory = std::function<std::shared_ptr<compute::SinkNodeConsumer>()>;
+
+ARROW_ENGINE_EXPORT Result<std::vector<compute::Declaration>> DeserializePlan(
+    const Buffer&, const ConsumerFactory&, ExtensionSet* ext_set = NULLPTR);
+
+ARROW_ENGINE_EXPORT
+Result<std::shared_ptr<DataType>> DeserializeType(const Buffer&, const ExtensionSet&);
+
+ARROW_ENGINE_EXPORT
+Result<std::shared_ptr<Buffer>> SerializeType(const DataType&, ExtensionSet*);
+
+ARROW_ENGINE_EXPORT
+Result<std::shared_ptr<Schema>> DeserializeSchema(const Buffer&, const ExtensionSet&);
+
+ARROW_ENGINE_EXPORT
+Result<std::shared_ptr<Buffer>> SerializeSchema(const Schema&, ExtensionSet*);
+
+ARROW_ENGINE_EXPORT
+Result<compute::Expression> DeserializeExpression(const Buffer&, const ExtensionSet&);
+
+ARROW_ENGINE_EXPORT
+Result<std::shared_ptr<Buffer>> SerializeExpression(const compute::Expression&,
+                                                    ExtensionSet*);

Review comment:
       What's the use case for serializing?  Is that for testing and internal use only?

##########
File path: cpp/src/arrow/engine/substrait/serde.h
##########
@@ -0,0 +1,79 @@
+// 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 <string>
+#include <vector>
+
+#include "arrow/buffer.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/engine/substrait/extension_set.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+
+using ConsumerFactory = std::function<std::shared_ptr<compute::SinkNodeConsumer>()>;
+
+ARROW_ENGINE_EXPORT Result<std::vector<compute::Declaration>> DeserializePlan(

Review comment:
       It doesn't have to be done in this PR but, as this is probably the primary user interface to the consumer, we should document the methods in this file.  Feel free to file a follow-up JIRA.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.cc
##########
@@ -0,0 +1,902 @@
+// 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.
+
+#include "arrow/engine/substrait/expression_internal.h"
+
+#include <utility>
+
+#include "arrow/builder.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/exec/expression.h"
+#include "arrow/compute/exec/expression_internal.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/make_unique.h"
+#include "arrow/visit_scalar_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace engine {
+
+namespace internal {
+using ::arrow::internal::make_unique;
+}  // namespace internal
+
+namespace {
+
+std::shared_ptr<FixedSizeBinaryScalar> FixedSizeBinaryScalarFromBytes(
+    const std::string& bytes) {
+  auto buf = Buffer::FromString(bytes);
+  auto type = fixed_size_binary(static_cast<int>(buf->size()));
+  return std::make_shared<FixedSizeBinaryScalar>(std::move(buf), std::move(type));
+}
+
+}  // namespace
+
+Result<compute::Expression> FromProto(const substrait::Expression& expr,
+                                      const ExtensionSet& ext_set) {
+  switch (expr.rex_type_case()) {
+    case substrait::Expression::kLiteral: {
+      ARROW_ASSIGN_OR_RAISE(auto datum, FromProto(expr.literal(), ext_set));
+      return compute::literal(std::move(datum));
+    }
+
+    case substrait::Expression::kSelection: {
+      if (!expr.selection().has_direct_reference()) break;
+
+      util::optional<compute::Expression> out;
+      if (expr.selection().has_expression()) {
+        ARROW_ASSIGN_OR_RAISE(out, FromProto(expr.selection().expression(), ext_set));
+      }
+
+      const auto* ref = &expr.selection().direct_reference();
+      while (ref != nullptr) {
+        switch (ref->reference_type_case()) {
+          case substrait::Expression::ReferenceSegment::kStructField: {
+            auto index = ref->struct_field().field();
+            if (!out) {
+              // Root StructField (column selection)
+              out = compute::field_ref(FieldRef(index));
+            } else if (auto out_ref = out->field_ref()) {
+              // Nested StructFields on the root (selection of struct-typed column
+              // combined with selecting struct fields)
+              out = compute::field_ref(FieldRef(*out_ref, index));
+            } else if (out->call() && out->call()->function_name == "struct_field") {
+              // Nested StructFields on top of an arbitrary expression
+              std::static_pointer_cast<arrow::compute::StructFieldOptions>(
+                  out->call()->options)
+                  ->indices.push_back(index);
+            } else {
+              // First StructField on top of an arbitrary expression
+              out = compute::call("struct_field", {std::move(*out)},
+                                  arrow::compute::StructFieldOptions({index}));
+            }
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->struct_field().has_child()) {
+              ref = &ref->struct_field().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          case substrait::Expression::ReferenceSegment::kListElement: {
+            if (!out) {
+              // Root ListField (illegal)
+              return Status::Invalid(
+                  "substrait::ListElement cannot take a Relation as an argument");
+            }
+
+            // ListField on top of an arbitrary expression
+            out = compute::call(
+                "list_element",
+                {std::move(*out), compute::literal(ref->list_element().offset())});
+
+            // Segment handled, continue with child segment (if any)
+            if (ref->list_element().has_child()) {
+              ref = &ref->list_element().child();
+            } else {
+              ref = nullptr;
+            }
+            break;
+          }
+          default:
+            // Unimplemented construct, break out of loop
+            out.reset();
+            ref = nullptr;
+        }
+      }
+      if (out) {
+        return *std::move(out);
+      }
+      break;
+    }
+
+    case substrait::Expression::kIfThen: {
+      const auto& if_then = expr.if_then();
+      if (!if_then.has_else_()) break;
+      if (if_then.ifs_size() == 0) break;
+
+      if (if_then.ifs_size() == 1) {
+        ARROW_ASSIGN_OR_RAISE(auto if_, FromProto(if_then.ifs(0).if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto then, FromProto(if_then.ifs(0).then(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto else_, FromProto(if_then.else_(), ext_set));
+        return compute::call("if_else",
+                             {std::move(if_), std::move(then), std::move(else_)});
+      }
+
+      std::vector<compute::Expression> conditions, args;
+      std::vector<std::string> condition_names;
+      conditions.reserve(if_then.ifs_size());
+      condition_names.reserve(if_then.ifs_size());
+      size_t name_counter = 0;
+      args.reserve(if_then.ifs_size() + 2);
+      args.emplace_back();
+      for (auto if_ : if_then.ifs()) {
+        ARROW_ASSIGN_OR_RAISE(auto compute_if, FromProto(if_.if_(), ext_set));
+        ARROW_ASSIGN_OR_RAISE(auto compute_then, FromProto(if_.then(), ext_set));
+        conditions.emplace_back(std::move(compute_if));
+        args.emplace_back(std::move(compute_then));
+        condition_names.emplace_back("cond" + std::to_string(++name_counter));
+      }
+      ARROW_ASSIGN_OR_RAISE(auto compute_else, FromProto(if_then.else_(), ext_set));
+      args.emplace_back(std::move(compute_else));
+      args[0] = compute::call("make_struct", std::move(conditions),
+                              compute::MakeStructOptions(condition_names));
+      return compute::call("case_when", std::move(args));
+    }
+
+    case substrait::Expression::kScalarFunction: {
+      const auto& scalar_fn = expr.scalar_function();
+
+      auto id = ext_set.function_ids()[scalar_fn.function_reference()];
+
+      std::vector<compute::Expression> arguments(scalar_fn.args_size());
+      for (int i = 0; i < scalar_fn.args_size(); ++i) {
+        ARROW_ASSIGN_OR_RAISE(arguments[i], FromProto(scalar_fn.args(i), ext_set));
+      }
+
+      return compute::call(id.name.to_string(), std::move(arguments));
+    }
+
+    default:
+      break;
+  }
+
+  return Status::NotImplemented("conversion to arrow::compute::Expression from ",

Review comment:
       There's a number of features in the Substrait proto that we don't support yet (e.g. switch_expression, single_or_list, multi_or_list).  I'm not recommending we handle those as part of this PR (it is plenty big enough) but I wonder how we track what portion of the spec we've implemented so far?

##########
File path: cpp/src/arrow/engine/substrait/relation_internal.cc
##########
@@ -0,0 +1,186 @@
+// 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/engine/substrait/relation_internal.h"
+
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/dataset/plan.h"
+#include "arrow/dataset/scanner.h"
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/filesystem/localfs.h"
+
+namespace arrow {
+namespace engine {
+
+template <typename RelMessage>
+Status CheckRelCommon(const RelMessage& rel) {
+  if (rel.has_common()) {
+    if (rel.common().has_emit()) {
+      return Status::NotImplemented("substrait::RelCommon::Emit");
+    }
+    if (rel.common().has_hint()) {
+      return Status::NotImplemented("substrait::RelCommon::Hint");
+    }

Review comment:
       Seems like we could ignore this.

##########
File path: cpp/src/arrow/engine/substrait/expression_internal.h
##########
@@ -0,0 +1,49 @@
+// 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 <utility>
+
+#include "arrow/compute/type_fwd.h"
+#include "arrow/engine/substrait/extension_set.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/type_fwd.h"
+
+#include "substrait/expression.pb.h"  // IWYU pragma: export
+
+namespace arrow {
+namespace engine {
+
+ARROW_ENGINE_EXPORT
+Result<compute::Expression> FromProto(const substrait::Expression&, const ExtensionSet&);
+
+ARROW_ENGINE_EXPORT
+Result<std::unique_ptr<substrait::Expression>> ToProto(const compute::Expression&,
+                                                       ExtensionSet*);
+
+ARROW_ENGINE_EXPORT
+Result<Datum> FromProto(const substrait::Expression::Literal&, const ExtensionSet&);
+
+ARROW_ENGINE_EXPORT
+Result<std::unique_ptr<substrait::Expression::Literal>> ToProto(const Datum&,
+                                                                ExtensionSet*);

Review comment:
       Super minor nit:  I kind of prefer `FromProtoLiteral`, `FromProtoExpression`, `ToProtoLiteral`, and `ToProtoExpression` vs. the overloaded names.  

##########
File path: cpp/src/arrow/engine/substrait/relation_internal.cc
##########
@@ -0,0 +1,186 @@
+// 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/engine/substrait/relation_internal.h"
+
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/dataset/plan.h"
+#include "arrow/dataset/scanner.h"
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/filesystem/localfs.h"
+
+namespace arrow {
+namespace engine {
+
+template <typename RelMessage>
+Status CheckRelCommon(const RelMessage& rel) {
+  if (rel.has_common()) {
+    if (rel.common().has_emit()) {
+      return Status::NotImplemented("substrait::RelCommon::Emit");
+    }
+    if (rel.common().has_hint()) {
+      return Status::NotImplemented("substrait::RelCommon::Hint");
+    }
+    if (rel.common().has_advanced_extension()) {
+      return Status::NotImplemented("substrait::RelCommon::advanced_extension");
+    }
+  }
+  if (rel.has_advanced_extension()) {
+    return Status::NotImplemented("substrait AdvancedExtensions");
+  }
+  return Status::OK();
+}
+
+Result<compute::Declaration> FromProto(const substrait::Rel& rel,
+                                       const ExtensionSet& ext_set) {
+  static bool dataset_init = false;
+  if (!dataset_init) {
+    dataset_init = true;
+    dataset::internal::Initialize();
+  }
+
+  switch (rel.rel_type_case()) {
+    case substrait::Rel::RelTypeCase::kRead: {
+      const auto& read = rel.read();
+      RETURN_NOT_OK(CheckRelCommon(read));
+
+      ARROW_ASSIGN_OR_RAISE(auto base_schema, FromProto(read.base_schema(), ext_set));
+
+      auto scan_options = std::make_shared<dataset::ScanOptions>();
+
+      if (read.has_filter()) {
+        ARROW_ASSIGN_OR_RAISE(scan_options->filter, FromProto(read.filter(), ext_set));
+      }
+
+      if (read.has_projection()) {
+        return Status::NotImplemented("substrait::ReadRel::projection");
+      }

Review comment:
       Let's add a JIRA for this.

##########
File path: cpp/src/arrow/engine/substrait/relation_internal.cc
##########
@@ -0,0 +1,186 @@
+// 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/engine/substrait/relation_internal.h"
+
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/dataset/plan.h"
+#include "arrow/dataset/scanner.h"
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/filesystem/localfs.h"
+
+namespace arrow {
+namespace engine {
+
+template <typename RelMessage>
+Status CheckRelCommon(const RelMessage& rel) {
+  if (rel.has_common()) {
+    if (rel.common().has_emit()) {
+      return Status::NotImplemented("substrait::RelCommon::Emit");
+    }

Review comment:
       This is basically a project right?  Let's add a JIRA for this.

##########
File path: cpp/src/arrow/engine/substrait/extension_set.cc
##########
@@ -0,0 +1,346 @@
+// 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/engine/substrait/extension_set.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/util/hash_util.h"
+#include "arrow/util/hashing.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+namespace {
+
+struct TypePtrHashEq {
+  template <typename Ptr>
+  size_t operator()(const Ptr& type) const {
+    return type->Hash();
+  }
+
+  template <typename Ptr>
+  bool operator()(const Ptr& l, const Ptr& r) const {
+    return *l == *r;
+  }
+};
+
+struct IdHashEq {
+  using Id = ExtensionSet::Id;
+
+  size_t operator()(Id id) const {
+    constexpr ::arrow::internal::StringViewHash hash = {};
+    auto out = static_cast<size_t>(hash(id.uri));
+    ::arrow::internal::hash_combine(out, hash(id.name));
+    return out;
+  }
+
+  bool operator()(Id l, Id r) const { return l.uri == r.uri && l.name == r.name; }
+};
+
+}  // namespace
+
+struct ExtensionSet::Impl {
+  void AddUri(util::string_view uri, ExtensionSet* self) {
+    if (uris_.find(uri) != uris_.end()) return;
+
+    self->uris_.push_back(uri);
+    uris_.insert(self->uris_.back());  // lookup helper's keys should reference memory
+                                       // owned by this ExtensionSet
+  }
+
+  Status CheckHasUri(util::string_view uri) {
+    if (uris_.find(uri) != uris_.end()) return Status::OK();
+
+    return Status::Invalid(
+        "Uri ", uri,
+        " was referenced by an extension but was not declared in the ExtensionSet.");
+  }
+
+  uint32_t EncodeType(Id id, const std::shared_ptr<DataType>& type, bool is_variation,
+                      ExtensionSet* self) {
+    // note: at this point we're guaranteed to have an Id which points to memory owned by
+    // the set's registry.
+    AddUri(id.uri, self);
+    auto it_success = types_.emplace(id, static_cast<uint32_t>(types_.size()));
+
+    if (it_success.second) {
+      DCHECK_EQ(self->type_ids_.size(), self->types_.size());
+      self->type_ids_.push_back(id);
+      self->types_.push_back(type);
+      self->type_is_variation_.push_back(is_variation);
+    }
+
+    return it_success.first->second;
+  }
+
+  uint32_t EncodeFunction(Id id, util::string_view function_name, ExtensionSet* self) {
+    // note: at this point we're guaranteed to have an Id which points to memory owned by
+    // the set's registry.
+    AddUri(id.uri, self);
+    auto it_success = functions_.emplace(id, static_cast<uint32_t>(functions_.size()));
+
+    if (it_success.second) {
+      DCHECK_EQ(self->function_ids_.size(), self->function_names_.size());
+      self->function_ids_.push_back(id);
+      self->function_names_.push_back(function_name);
+    }
+
+    return it_success.first->second;
+  }
+
+  std::unordered_set<util::string_view, ::arrow::internal::StringViewHash> uris_;
+  std::unordered_map<Id, uint32_t, IdHashEq, IdHashEq> types_, functions_;
+};
+
+ExtensionSet::ExtensionSet(ExtensionIdRegistry* registry)
+    : registry_(registry), impl_(new Impl(), [](Impl* impl) { delete impl; }) {}
+
+Result<ExtensionSet> ExtensionSet::Make(std::vector<util::string_view> uris,
+                                        std::vector<Id> type_ids,
+                                        std::vector<bool> type_is_variation,
+                                        std::vector<Id> function_ids,
+                                        ExtensionIdRegistry* registry) {
+  ExtensionSet set;
+  set.registry_ = registry;
+
+  // TODO(bkietz) move this into the registry as registry->OwnUris(&uris) or so
+  std::unordered_set<util::string_view, ::arrow::internal::StringViewHash>
+      uris_owned_by_registry;
+  for (util::string_view uri : registry->Uris()) {
+    uris_owned_by_registry.insert(uri);
+  }
+
+  for (auto& uri : uris) {
+    if (uri.empty()) continue;
+    auto it = uris_owned_by_registry.find(uri);
+    if (it == uris_owned_by_registry.end()) {
+      return Status::KeyError("Uri '", uri, "' not found in registry");
+    }
+    uri = *it;  // Ensure uris point into the registry's memory
+    set.impl_->AddUri(*it, &set);
+  }
+
+  if (type_ids.size() != type_is_variation.size()) {
+    return Status::Invalid("Received ", type_ids.size(), " type ids but a ",
+                           type_is_variation.size(), "-long is_variation vector");
+  }
+
+  set.types_.resize(type_ids.size());
+
+  for (size_t i = 0; i < type_ids.size(); ++i) {
+    if (type_ids[i].empty()) continue;
+    RETURN_NOT_OK(set.impl_->CheckHasUri(type_ids[i].uri));
+
+    if (auto rec = registry->GetType(type_ids[i], type_is_variation[i])) {
+      set.types_[i] = rec->type;
+      type_ids[i] = rec->id;  // use Id which references memory owned by the registry
+      continue;
+    }
+    return Status::Invalid("Type", (type_is_variation[i] ? " variation" : ""), " ",
+                           type_ids[i].uri, "#", type_ids[i].name, " not found");
+  }
+
+  set.function_names_.resize(function_ids.size());
+
+  for (size_t i = 0; i < function_ids.size(); ++i) {
+    if (function_ids[i].empty()) continue;
+    RETURN_NOT_OK(set.impl_->CheckHasUri(function_ids[i].uri));
+
+    if (auto rec = registry->GetFunction(function_ids[i])) {
+      set.function_names_[i] = rec->function_name;
+      function_ids[i] = rec->id;  // use Id which references memory owned by the registry
+      continue;
+    }
+    return Status::Invalid("Function ", function_ids[i].uri, "#", type_ids[i].name,
+                           " not found");
+  }
+
+  set.function_ids_ = std::move(function_ids);
+  set.type_ids_ = std::move(type_ids);
+  set.type_is_variation_ = std::move(type_is_variation);
+  set.uris_ = std::move(uris);
+
+  return std::move(set);
+}
+
+Result<uint32_t> ExtensionSet::EncodeType(const DataType& type) {
+  if (auto rec = registry_->GetType(type)) {
+    return impl_->EncodeType(rec->id, rec->type, rec->is_variation, this);
+  }
+  return Status::KeyError("type ", type.ToString(), " not found in the registry");
+}
+
+Result<uint32_t> ExtensionSet::EncodeFunction(util::string_view function_name) {
+  if (auto rec = registry_->GetFunction(function_name)) {
+    return impl_->EncodeFunction(rec->id, rec->function_name, this);
+  }
+  return Status::KeyError("function ", function_name, " not found in the registry");
+}
+
+template <typename KeyToIndex, typename Key>
+const int* GetIndex(const KeyToIndex& key_to_index, const Key& key) {
+  auto it = key_to_index.find(key);
+  if (it == key_to_index.end()) return nullptr;
+  return &it->second;
+}
+
+ExtensionIdRegistry* default_extension_id_registry() {
+  static struct Impl : ExtensionIdRegistry {
+    Impl() {
+      struct TypeName {

Review comment:
       Nit: `TypeAlias`?

##########
File path: cpp/src/arrow/engine/substrait/relation_internal.cc
##########
@@ -0,0 +1,186 @@
+// 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/engine/substrait/relation_internal.h"
+
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/dataset/plan.h"
+#include "arrow/dataset/scanner.h"
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/filesystem/localfs.h"
+
+namespace arrow {
+namespace engine {
+
+template <typename RelMessage>
+Status CheckRelCommon(const RelMessage& rel) {
+  if (rel.has_common()) {
+    if (rel.common().has_emit()) {
+      return Status::NotImplemented("substrait::RelCommon::Emit");
+    }
+    if (rel.common().has_hint()) {
+      return Status::NotImplemented("substrait::RelCommon::Hint");
+    }
+    if (rel.common().has_advanced_extension()) {
+      return Status::NotImplemented("substrait::RelCommon::advanced_extension");
+    }
+  }
+  if (rel.has_advanced_extension()) {
+    return Status::NotImplemented("substrait AdvancedExtensions");
+  }
+  return Status::OK();
+}
+
+Result<compute::Declaration> FromProto(const substrait::Rel& rel,
+                                       const ExtensionSet& ext_set) {
+  static bool dataset_init = false;
+  if (!dataset_init) {
+    dataset_init = true;
+    dataset::internal::Initialize();
+  }
+
+  switch (rel.rel_type_case()) {
+    case substrait::Rel::RelTypeCase::kRead: {
+      const auto& read = rel.read();
+      RETURN_NOT_OK(CheckRelCommon(read));
+
+      ARROW_ASSIGN_OR_RAISE(auto base_schema, FromProto(read.base_schema(), ext_set));
+
+      auto scan_options = std::make_shared<dataset::ScanOptions>();
+
+      if (read.has_filter()) {
+        ARROW_ASSIGN_OR_RAISE(scan_options->filter, FromProto(read.filter(), ext_set));
+      }
+
+      if (read.has_projection()) {
+        return Status::NotImplemented("substrait::ReadRel::projection");
+      }
+
+      {
+        // just project all fields
+        std::vector<compute::Expression> expressions{base_schema->fields().size()};
+        for (int i = 0; i < base_schema->num_fields(); ++i) {
+          expressions[i] = compute::field_ref(i);
+        }
+        scan_options->projection =
+            compute::call("make_struct", std::move(expressions),
+                          compute::MakeStructOptions{base_schema->field_names()});

Review comment:
       This is harmless but also completely extraneous.  `scan_options->projection` is only accessed when using one of the scanner "lightweight producer" functions like "ToTable" or "Head".  It isn't referenced by the ScanNode at all.

##########
File path: cpp/src/arrow/engine/substrait/relation_internal.cc
##########
@@ -0,0 +1,186 @@
+// 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/engine/substrait/relation_internal.h"
+
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/dataset/plan.h"
+#include "arrow/dataset/scanner.h"
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/filesystem/localfs.h"
+
+namespace arrow {
+namespace engine {
+
+template <typename RelMessage>
+Status CheckRelCommon(const RelMessage& rel) {
+  if (rel.has_common()) {
+    if (rel.common().has_emit()) {
+      return Status::NotImplemented("substrait::RelCommon::Emit");
+    }
+    if (rel.common().has_hint()) {
+      return Status::NotImplemented("substrait::RelCommon::Hint");
+    }
+    if (rel.common().has_advanced_extension()) {
+      return Status::NotImplemented("substrait::RelCommon::advanced_extension");
+    }
+  }
+  if (rel.has_advanced_extension()) {
+    return Status::NotImplemented("substrait AdvancedExtensions");
+  }
+  return Status::OK();
+}
+
+Result<compute::Declaration> FromProto(const substrait::Rel& rel,
+                                       const ExtensionSet& ext_set) {
+  static bool dataset_init = false;
+  if (!dataset_init) {
+    dataset_init = true;
+    dataset::internal::Initialize();
+  }
+
+  switch (rel.rel_type_case()) {
+    case substrait::Rel::RelTypeCase::kRead: {
+      const auto& read = rel.read();
+      RETURN_NOT_OK(CheckRelCommon(read));
+
+      ARROW_ASSIGN_OR_RAISE(auto base_schema, FromProto(read.base_schema(), ext_set));
+
+      auto scan_options = std::make_shared<dataset::ScanOptions>();
+
+      if (read.has_filter()) {
+        ARROW_ASSIGN_OR_RAISE(scan_options->filter, FromProto(read.filter(), ext_set));
+      }
+
+      if (read.has_projection()) {
+        return Status::NotImplemented("substrait::ReadRel::projection");
+      }
+
+      {
+        // just project all fields
+        std::vector<compute::Expression> expressions{base_schema->fields().size()};
+        for (int i = 0; i < base_schema->num_fields(); ++i) {
+          expressions[i] = compute::field_ref(i);
+        }
+        scan_options->projection =
+            compute::call("make_struct", std::move(expressions),
+                          compute::MakeStructOptions{base_schema->field_names()});
+      }
+
+      if (!read.has_local_files()) {
+        return Status::NotImplemented(
+            "substrait::ReadRel with read_type other than LocalFiles");
+      }
+
+      if (read.local_files().has_advanced_extension()) {
+        return Status::NotImplemented(
+            "substrait::ReadRel::LocalFiles::advanced_extension");
+      }
+
+      auto format = std::make_shared<dataset::ParquetFileFormat>();
+      auto filesystem = std::make_shared<fs::LocalFileSystem>();
+      std::vector<std::shared_ptr<dataset::FileFragment>> fragments;
+
+      for (const auto& item : read.local_files().items()) {
+        if (!item.has_uri_file()) {
+          return Status::NotImplemented(
+              "substrait::ReadRel::LocalFiles::FileOrFiles with "
+              "path_type other than uri_file");
+        }

Review comment:
       Let's add a JIRA for this.

##########
File path: cpp/src/arrow/engine/substrait/relation_internal.cc
##########
@@ -0,0 +1,186 @@
+// 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/engine/substrait/relation_internal.h"
+
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/dataset/plan.h"
+#include "arrow/dataset/scanner.h"
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/filesystem/localfs.h"
+
+namespace arrow {
+namespace engine {
+
+template <typename RelMessage>
+Status CheckRelCommon(const RelMessage& rel) {
+  if (rel.has_common()) {
+    if (rel.common().has_emit()) {
+      return Status::NotImplemented("substrait::RelCommon::Emit");
+    }
+    if (rel.common().has_hint()) {
+      return Status::NotImplemented("substrait::RelCommon::Hint");
+    }
+    if (rel.common().has_advanced_extension()) {
+      return Status::NotImplemented("substrait::RelCommon::advanced_extension");
+    }
+  }
+  if (rel.has_advanced_extension()) {
+    return Status::NotImplemented("substrait AdvancedExtensions");
+  }
+  return Status::OK();
+}
+
+Result<compute::Declaration> FromProto(const substrait::Rel& rel,
+                                       const ExtensionSet& ext_set) {
+  static bool dataset_init = false;
+  if (!dataset_init) {
+    dataset_init = true;
+    dataset::internal::Initialize();
+  }
+
+  switch (rel.rel_type_case()) {
+    case substrait::Rel::RelTypeCase::kRead: {
+      const auto& read = rel.read();
+      RETURN_NOT_OK(CheckRelCommon(read));
+
+      ARROW_ASSIGN_OR_RAISE(auto base_schema, FromProto(read.base_schema(), ext_set));
+
+      auto scan_options = std::make_shared<dataset::ScanOptions>();
+
+      if (read.has_filter()) {
+        ARROW_ASSIGN_OR_RAISE(scan_options->filter, FromProto(read.filter(), ext_set));
+      }
+
+      if (read.has_projection()) {
+        return Status::NotImplemented("substrait::ReadRel::projection");
+      }
+
+      {
+        // just project all fields
+        std::vector<compute::Expression> expressions{base_schema->fields().size()};
+        for (int i = 0; i < base_schema->num_fields(); ++i) {
+          expressions[i] = compute::field_ref(i);
+        }
+        scan_options->projection =
+            compute::call("make_struct", std::move(expressions),
+                          compute::MakeStructOptions{base_schema->field_names()});
+      }
+
+      if (!read.has_local_files()) {
+        return Status::NotImplemented(
+            "substrait::ReadRel with read_type other than LocalFiles");
+      }
+
+      if (read.local_files().has_advanced_extension()) {
+        return Status::NotImplemented(
+            "substrait::ReadRel::LocalFiles::advanced_extension");
+      }
+
+      auto format = std::make_shared<dataset::ParquetFileFormat>();
+      auto filesystem = std::make_shared<fs::LocalFileSystem>();
+      std::vector<std::shared_ptr<dataset::FileFragment>> fragments;
+
+      for (const auto& item : read.local_files().items()) {
+        if (!item.has_uri_file()) {
+          return Status::NotImplemented(
+              "substrait::ReadRel::LocalFiles::FileOrFiles with "
+              "path_type other than uri_file");
+        }
+
+        if (item.format() !=
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET) {
+          return Status::NotImplemented(
+              "substrait::ReadRel::LocalFiles::FileOrFiles::format "
+              "other than FILE_FORMAT_PARQUET");

Review comment:
       Let's add a JIRA for this.

##########
File path: cpp/src/arrow/engine/substrait/extension_set.h
##########
@@ -0,0 +1,137 @@
+// 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 <vector>
+
+#include "arrow/engine/visibility.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+
+/// A mapping from arrow types and functions to the (uri, name) which identifies
+/// the corresponding substrait extension. Substrait types and variations must be
+/// registered with their corresponding arrow::DataType before they can be used!
+class ARROW_ENGINE_EXPORT ExtensionIdRegistry {

Review comment:
       Maybe clarify what "Substrait types" means.  The stock set of types builtin to Substrait doesn't have to be registered with the extension id registry.  However, there are a number of Arrow types which aren't covered by Substrait's stock types and these must be registered.  In addition, there are some stock Substrait types  that don't have an Arrow equivalent (e.g. UUID) and so there are Arrow extension types for those but they have nothing to do with the extension ID registry (I don't think?)

##########
File path: cpp/src/arrow/engine/simple_extension_type_internal.h
##########
@@ -0,0 +1,183 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include "arrow/extension_type.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/reflection_internal.h"
+#include "arrow/util/string.h"
+
+namespace arrow {
+namespace engine {
+
+template <const util::string_view& kExtensionName, typename Params,
+          typename ParamsProperties, const ParamsProperties* kProperties,
+          std::shared_ptr<DataType> GetStorage(const Params&)>
+class SimpleExtensionType : public ExtensionType {
+ public:
+  using ParamsType = Params;

Review comment:
       Is this `using` statement needed?

##########
File path: cpp/src/arrow/engine/simple_extension_type_internal.h
##########
@@ -0,0 +1,183 @@
+// 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.
+
+#pragma once
+
+#include <memory>
+#include <sstream>
+#include <string>
+#include <vector>
+
+#include "arrow/extension_type.h"
+#include "arrow/util/logging.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/reflection_internal.h"
+#include "arrow/util/string.h"
+
+namespace arrow {
+namespace engine {
+
+template <const util::string_view& kExtensionName, typename Params,
+          typename ParamsProperties, const ParamsProperties* kProperties,
+          std::shared_ptr<DataType> GetStorage(const Params&)>
+class SimpleExtensionType : public ExtensionType {
+ public:
+  using ParamsType = Params;
+
+  explicit SimpleExtensionType(std::shared_ptr<DataType> storage_type, Params params = {})
+      : ExtensionType(std::move(storage_type)), params_(std::move(params)) {}
+
+  static std::shared_ptr<DataType> Make(Params params) {
+    auto storage_type = GetStorage(params);
+    return std::make_shared<SimpleExtensionType>(std::move(storage_type),
+                                                 std::move(params));
+  }
+
+  static const Params* GetIf(const DataType& type) {
+    if (type.id() != Type::EXTENSION) return nullptr;
+
+    const auto& ext_type = ::arrow::internal::checked_cast<const ExtensionType&>(type);
+    if (ext_type.extension_name() != kExtensionName) return nullptr;
+
+    return &::arrow::internal::checked_cast<const SimpleExtensionType&>(type).params_;
+  }
+
+  std::string extension_name() const override { return kExtensionName.to_string(); }
+
+  std::string ToString() const override { return "extension<" + this->Serialize() + ">"; }
+
+  struct ExtensionEqualsImpl {
+    ExtensionEqualsImpl(const Params& l, const Params& r) : left_(l), right_(r) {
+      kProperties->ForEach(*this);
+    }
+
+    template <typename Property>
+    void operator()(const Property& prop, size_t i) {
+      equal_ &= prop.get(left_) == prop.get(right_);
+    }
+
+    const Params& left_;
+    const Params& right_;
+    bool equal_ = true;
+  };
+  bool ExtensionEquals(const ExtensionType& other) const override {
+    if (kExtensionName != other.extension_name()) return false;
+    const auto& other_params = static_cast<const SimpleExtensionType&>(other).params_;
+    return ExtensionEqualsImpl(params_, other_params).equal_;
+  }
+
+  std::shared_ptr<Array> MakeArray(std::shared_ptr<ArrayData> data) const override {
+    DCHECK_EQ(data->type->id(), Type::EXTENSION);
+    DCHECK_EQ(static_cast<const ExtensionType&>(*data->type).extension_name(),

Review comment:
       ```suggestion
       DCHECK_EQ(arrow::internal::checked_pointer_cast<ExtensionType>(data->type)->extension_name(),
   ```
   I'm a little out of my depth here but this `static_cast` seems odd.  It's entirely possible that it's just fine.  I suppose with a checked_pointer_cast you should probably check to make sure it isn't nullptr.

##########
File path: cpp/src/arrow/engine/substrait/extension_set.h
##########
@@ -0,0 +1,137 @@
+// 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 <vector>
+
+#include "arrow/engine/visibility.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+
+/// A mapping from arrow types and functions to the (uri, name) which identifies
+/// the corresponding substrait extension. Substrait types and variations must be
+/// registered with their corresponding arrow::DataType before they can be used!
+class ARROW_ENGINE_EXPORT ExtensionIdRegistry {
+ public:
+  /// All uris registered in this ExtensionIdRegistry
+  virtual std::vector<util::string_view> Uris() const = 0;
+
+  struct Id {
+    util::string_view uri, name;
+
+    bool empty() const { return uri.empty() && name.empty(); }
+  };
+
+  struct TypeRecord {
+    Id id;
+    const std::shared_ptr<DataType>& type;
+    bool is_variation;
+  };
+  virtual util::optional<TypeRecord> GetType(const DataType&) const = 0;
+  virtual util::optional<TypeRecord> GetType(Id, bool is_variation) const = 0;
+  virtual Status RegisterType(Id, std::shared_ptr<DataType>, bool is_variation) = 0;
+
+  // FIXME some functions will not be simple enough to convert without access to their
+  // arguments/options. For example is_in embeds the set in options rather than using an
+  // argument:
+  //     is_in(x, SetLookupOptions(set)) <-> (k...Uri, "is_in")(x, set)
+  //
+  // ... for another example, depending on the value of the first argument to
+  // substrait::add it either corresponds to arrow::add or arrow::add_checked
+  struct FunctionRecord {
+    Id id;
+    const std::string& function_name;
+  };
+  virtual util::optional<FunctionRecord> GetFunction(Id) const = 0;
+  virtual util::optional<FunctionRecord> GetFunction(
+      util::string_view arrow_function_name) const = 0;
+  virtual Status RegisterFunction(Id, std::string arrow_function_name) = 0;
+};
+
+constexpr util::string_view kArrowExtTypesUri =
+    "https://github.com/apache/arrow/blob/master/format/substrait/"
+    "extension_types.yaml";
+
+ARROW_ENGINE_EXPORT ExtensionIdRegistry* default_extension_id_registry();
+
+/// A subset of an ExtensionIdRegistry with extensions identifiable by an integer.
+///
+/// ExtensionSet does not own strings; it only refers to strings in an
+/// ExtensionIdRegistry.
+class ARROW_ENGINE_EXPORT ExtensionSet {
+ public:
+  using Id = ExtensionIdRegistry::Id;
+
+  /// Construct an empty ExtensionSet to be populated during serialization.
+  explicit ExtensionSet(ExtensionIdRegistry* = default_extension_id_registry());
+  ARROW_DEFAULT_MOVE_AND_ASSIGN(ExtensionSet);
+
+  /// Construct an ExtensionSet with explicit extension ids for efficient referencing
+  /// during deserialization. Note that input vectors need not be densely packed; an empty
+  /// (default constructed) Id may be used as a placeholder to indicate an unused
+  /// _anchor/_reference. This factory will be used to wrap the extensions declared in a
+  /// substrait::Plan before deserializing the plan's relations.
+  ///
+  /// Views will be replaced with equivalent views pointing to memory owned by the
+  /// registry.
+  static Result<ExtensionSet> Make(
+      std::vector<util::string_view> uris, std::vector<Id> type_ids,
+      std::vector<bool> type_is_variation, std::vector<Id> function_ids,
+      ExtensionIdRegistry* = default_extension_id_registry());
+
+  // index in these vectors == value of _anchor/_reference fields
+  /// FIXME this assumes that _anchor/_references won't be huge, which is not guaranteed.
+  /// Could it be?
+  const std::vector<util::string_view>& uris() const { return uris_; }

Review comment:
       It certainly could be.  I wouldn't expect a producer to do so but it seems too likely for comfort.  Maybe we can push for Substrait to recommend low-valued anchors?

##########
File path: cpp/src/arrow/engine/substrait/extension_set.cc
##########
@@ -0,0 +1,346 @@
+// 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/engine/substrait/extension_set.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/util/hash_util.h"
+#include "arrow/util/hashing.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+namespace {
+
+struct TypePtrHashEq {
+  template <typename Ptr>
+  size_t operator()(const Ptr& type) const {
+    return type->Hash();
+  }
+
+  template <typename Ptr>
+  bool operator()(const Ptr& l, const Ptr& r) const {
+    return *l == *r;
+  }
+};
+
+struct IdHashEq {
+  using Id = ExtensionSet::Id;
+
+  size_t operator()(Id id) const {
+    constexpr ::arrow::internal::StringViewHash hash = {};
+    auto out = static_cast<size_t>(hash(id.uri));
+    ::arrow::internal::hash_combine(out, hash(id.name));
+    return out;
+  }
+
+  bool operator()(Id l, Id r) const { return l.uri == r.uri && l.name == r.name; }
+};
+
+}  // namespace
+
+struct ExtensionSet::Impl {
+  void AddUri(util::string_view uri, ExtensionSet* self) {
+    if (uris_.find(uri) != uris_.end()) return;

Review comment:
       I find it hard to follow that `ExtensionSet::uris_` is a `std::vector` and `ExtensionSet::Impl::uris_` is a std::unordered_map`.  Can we use different variable names?

##########
File path: cpp/src/arrow/engine/substrait/relation_internal.cc
##########
@@ -0,0 +1,186 @@
+// 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/engine/substrait/relation_internal.h"
+
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/dataset/file_parquet.h"
+#include "arrow/dataset/plan.h"
+#include "arrow/dataset/scanner.h"
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/filesystem/localfs.h"
+
+namespace arrow {
+namespace engine {
+
+template <typename RelMessage>
+Status CheckRelCommon(const RelMessage& rel) {
+  if (rel.has_common()) {
+    if (rel.common().has_emit()) {
+      return Status::NotImplemented("substrait::RelCommon::Emit");
+    }
+    if (rel.common().has_hint()) {
+      return Status::NotImplemented("substrait::RelCommon::Hint");
+    }
+    if (rel.common().has_advanced_extension()) {
+      return Status::NotImplemented("substrait::RelCommon::advanced_extension");
+    }
+  }
+  if (rel.has_advanced_extension()) {
+    return Status::NotImplemented("substrait AdvancedExtensions");
+  }
+  return Status::OK();
+}
+
+Result<compute::Declaration> FromProto(const substrait::Rel& rel,
+                                       const ExtensionSet& ext_set) {
+  static bool dataset_init = false;
+  if (!dataset_init) {
+    dataset_init = true;
+    dataset::internal::Initialize();
+  }
+
+  switch (rel.rel_type_case()) {
+    case substrait::Rel::RelTypeCase::kRead: {
+      const auto& read = rel.read();
+      RETURN_NOT_OK(CheckRelCommon(read));
+
+      ARROW_ASSIGN_OR_RAISE(auto base_schema, FromProto(read.base_schema(), ext_set));
+
+      auto scan_options = std::make_shared<dataset::ScanOptions>();
+
+      if (read.has_filter()) {
+        ARROW_ASSIGN_OR_RAISE(scan_options->filter, FromProto(read.filter(), ext_set));
+      }
+
+      if (read.has_projection()) {
+        return Status::NotImplemented("substrait::ReadRel::projection");
+      }
+
+      {
+        // just project all fields
+        std::vector<compute::Expression> expressions{base_schema->fields().size()};
+        for (int i = 0; i < base_schema->num_fields(); ++i) {
+          expressions[i] = compute::field_ref(i);
+        }
+        scan_options->projection =
+            compute::call("make_struct", std::move(expressions),
+                          compute::MakeStructOptions{base_schema->field_names()});
+      }
+
+      if (!read.has_local_files()) {
+        return Status::NotImplemented(
+            "substrait::ReadRel with read_type other than LocalFiles");
+      }
+
+      if (read.local_files().has_advanced_extension()) {
+        return Status::NotImplemented(
+            "substrait::ReadRel::LocalFiles::advanced_extension");
+      }
+
+      auto format = std::make_shared<dataset::ParquetFileFormat>();
+      auto filesystem = std::make_shared<fs::LocalFileSystem>();
+      std::vector<std::shared_ptr<dataset::FileFragment>> fragments;
+
+      for (const auto& item : read.local_files().items()) {
+        if (!item.has_uri_file()) {
+          return Status::NotImplemented(
+              "substrait::ReadRel::LocalFiles::FileOrFiles with "
+              "path_type other than uri_file");
+        }
+
+        if (item.format() !=
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET) {
+          return Status::NotImplemented(
+              "substrait::ReadRel::LocalFiles::FileOrFiles::format "
+              "other than FILE_FORMAT_PARQUET");
+        }
+
+        if (!util::string_view{item.uri_file()}.starts_with("file:///")) {
+          return Status::NotImplemented(
+              "substrait::ReadRel::LocalFiles::FileOrFiles::uri_file "
+              "with other than local filesystem (file:///)");
+        }

Review comment:
       Let's add a JIRA for this (uri_folder in particular, I have no idea what uri_path is and uri_path_glob seems lower priority)

##########
File path: cpp/src/arrow/engine/substrait/extension_types.cc
##########
@@ -0,0 +1,151 @@
+// 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/engine/substrait/extension_types.h"
+
+#include <unordered_map>
+#include <unordered_set>

Review comment:
       ```suggestion
   ```

##########
File path: cpp/src/arrow/engine/substrait/extension_types.cc
##########
@@ -0,0 +1,151 @@
+// 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/engine/substrait/extension_types.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/engine/simple_extension_type_internal.h"
+#include "arrow/util/hash_util.h"

Review comment:
       ```suggestion
   ```

##########
File path: cpp/src/arrow/engine/substrait/extension_set.h
##########
@@ -0,0 +1,137 @@
+// 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 <vector>
+
+#include "arrow/engine/visibility.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+
+/// A mapping from arrow types and functions to the (uri, name) which identifies
+/// the corresponding substrait extension. Substrait types and variations must be
+/// registered with their corresponding arrow::DataType before they can be used!
+class ARROW_ENGINE_EXPORT ExtensionIdRegistry {
+ public:
+  /// All uris registered in this ExtensionIdRegistry
+  virtual std::vector<util::string_view> Uris() const = 0;
+
+  struct Id {
+    util::string_view uri, name;
+
+    bool empty() const { return uri.empty() && name.empty(); }
+  };
+
+  struct TypeRecord {
+    Id id;
+    const std::shared_ptr<DataType>& type;
+    bool is_variation;
+  };
+  virtual util::optional<TypeRecord> GetType(const DataType&) const = 0;
+  virtual util::optional<TypeRecord> GetType(Id, bool is_variation) const = 0;
+  virtual Status RegisterType(Id, std::shared_ptr<DataType>, bool is_variation) = 0;
+
+  // FIXME some functions will not be simple enough to convert without access to their

Review comment:
       Let's create a JIRA for this

##########
File path: cpp/src/arrow/engine/substrait/extension_set.h
##########
@@ -0,0 +1,137 @@
+// 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 <vector>
+
+#include "arrow/engine/visibility.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+
+/// A mapping from arrow types and functions to the (uri, name) which identifies
+/// the corresponding substrait extension. Substrait types and variations must be
+/// registered with their corresponding arrow::DataType before they can be used!
+class ARROW_ENGINE_EXPORT ExtensionIdRegistry {
+ public:
+  /// All uris registered in this ExtensionIdRegistry

Review comment:
       I might just be slow on the uptake but it took me a few seconds to realize why the `ExtensionIdRegistry` needed to own URIs.  Maybe add something like...
   
   ```
   Note: These URIs are referenced by the views in each Id
   ```

##########
File path: cpp/src/arrow/engine/substrait/serde.h
##########
@@ -0,0 +1,79 @@
+// 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 <string>
+#include <vector>
+
+#include "arrow/buffer.h"
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/compute/exec/options.h"
+#include "arrow/engine/substrait/extension_set.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/result.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+
+using ConsumerFactory = std::function<std::shared_ptr<compute::SinkNodeConsumer>()>;

Review comment:
       I mentioned this elsewhere but now I understand it better.  I think we could name this `RelationConsumer` or something like that.  Or add some documentation.  It isn't clear at a first glance that Arrow ExecPlan's "sink" maps to a "relation".
   
   That being said, I don't know if this feels quite right.  Shouldn't a substrait plan include a "sink" relation for handling the data?  For example, the relation could be a "write" relation or it could be a "send out via flight" relation or it could be a "send IPC to some named Arrow C bridge".
   
   However, I suppose those would all be custom relations at the moment (Substrait really hasn't defined that part) so I think what we have here is fine.

##########
File path: cpp/src/arrow/engine/substrait/extension_types.h
##########
@@ -0,0 +1,82 @@
+// 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 <vector>
+
+#include "arrow/buffer.h"
+#include "arrow/compute/function.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/type_fwd.h"
+#include "arrow/util/optional.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+
+// arrow::ExtensionTypes are provided to wrap uuid, fixed_char, varchar, interval_year,
+// and interval_day which are first-class types in substrait but do not appear in
+// the arrow type system.
+//
+// Note that these are not automatically registered with arrow::RegisterExtensionType(),
+// which means among other things that serialization of these types to IPC would fail.

Review comment:
       Also, pretty much all compute functions are going to fail as well.  We can maybe add a long term JIRA to add more support for these extension types.

##########
File path: cpp/src/arrow/engine/substrait/serde.cc
##########
@@ -0,0 +1,232 @@
+// 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/engine/substrait/serde.h"
+
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/plan_internal.h"
+#include "arrow/engine/substrait/relation_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/util/string_view.h"
+
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/io/zero_copy_stream_impl_lite.h>
+#include <google/protobuf/message.h>
+#include <google/protobuf/util/json_util.h>
+#include <google/protobuf/util/message_differencer.h>
+#include <google/protobuf/util/type_resolver_util.h>
+
+namespace arrow {
+namespace engine {
+
+Status ParseFromBufferImpl(const Buffer& buf, const std::string& full_name,
+                           google::protobuf::Message* message) {
+  google::protobuf::io::ArrayInputStream buf_stream{buf.data(),
+                                                    static_cast<int>(buf.size())};
+
+  if (message->ParseFromZeroCopyStream(&buf_stream)) {
+    return Status::OK();
+  }
+  return Status::IOError("ParseFromZeroCopyStream failed for ", full_name);
+}
+
+template <typename Message>
+Result<Message> ParseFromBuffer(const Buffer& buf) {
+  Message message;
+  ARROW_RETURN_NOT_OK(
+      ParseFromBufferImpl(buf, Message::descriptor()->full_name(), &message));
+  return message;
+}
+
+Result<compute::Declaration> DeserializeRelation(const Buffer& buf,
+                                                 const ExtensionSet& ext_set) {
+  ARROW_ASSIGN_OR_RAISE(auto rel, ParseFromBuffer<substrait::Rel>(buf));
+  return FromProto(rel, ext_set);
+}
+
+Result<std::vector<compute::Declaration>> DeserializePlan(
+    const Buffer& buf, const ConsumerFactory& consumer_factory,
+    ExtensionSet* ext_set_out) {
+  ARROW_ASSIGN_OR_RAISE(auto plan, ParseFromBuffer<substrait::Plan>(buf));
+
+  ARROW_ASSIGN_OR_RAISE(auto ext_set, GetExtensionSetFromPlan(plan));
+
+  std::vector<compute::Declaration> sink_decls;
+  for (const substrait::PlanRel& plan_rel : plan.relations()) {
+    if (plan_rel.has_root()) {
+      return Status::NotImplemented("substrait::PlanRel with custom output field names");
+    }
+    ARROW_ASSIGN_OR_RAISE(auto decl, FromProto(plan_rel.rel(), ext_set));
+
+    // pipe each relation into a consuming_sink node
+    auto sink_decl = compute::Declaration::Sequence({
+        std::move(decl),
+        {"consuming_sink", compute::ConsumingSinkNodeOptions{consumer_factory()}},
+    });
+    sink_decls.push_back(std::move(sink_decl));
+  }
+
+  if (ext_set_out) {
+    *ext_set_out = std::move(ext_set);
+  }
+  return sink_decls;
+}
+
+Result<std::shared_ptr<Schema>> DeserializeSchema(const Buffer& buf,
+                                                  const ExtensionSet& ext_set) {
+  ARROW_ASSIGN_OR_RAISE(auto named_struct, ParseFromBuffer<substrait::NamedStruct>(buf));
+  return FromProto(named_struct, ext_set);
+}
+
+Result<std::shared_ptr<Buffer>> SerializeSchema(const Schema& schema,
+                                                ExtensionSet* ext_set) {
+  ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(schema, ext_set));
+  std::string serialized = named_struct->SerializeAsString();
+  return Buffer::FromString(std::move(serialized));
+}
+
+Result<std::shared_ptr<DataType>> DeserializeType(const Buffer& buf,
+                                                  const ExtensionSet& ext_set) {
+  ARROW_ASSIGN_OR_RAISE(auto type, ParseFromBuffer<substrait::Type>(buf));
+  ARROW_ASSIGN_OR_RAISE(auto type_nullable, FromProto(type, ext_set));
+  return std::move(type_nullable.first);
+}
+
+Result<std::shared_ptr<Buffer>> SerializeType(const DataType& type,
+                                              ExtensionSet* ext_set) {
+  ARROW_ASSIGN_OR_RAISE(auto st_type, ToProto(type, /*nullable=*/true, ext_set));
+  std::string serialized = st_type->SerializeAsString();
+  return Buffer::FromString(std::move(serialized));
+}
+
+Result<compute::Expression> DeserializeExpression(const Buffer& buf,
+                                                  const ExtensionSet& ext_set) {
+  ARROW_ASSIGN_OR_RAISE(auto expr, ParseFromBuffer<substrait::Expression>(buf));
+  return FromProto(expr, ext_set);
+}
+
+Result<std::shared_ptr<Buffer>> SerializeExpression(const compute::Expression& expr,
+                                                    ExtensionSet* ext_set) {
+  ARROW_ASSIGN_OR_RAISE(auto st_expr, ToProto(expr, ext_set));
+  std::string serialized = st_expr->SerializeAsString();
+  return Buffer::FromString(std::move(serialized));
+}
+
+namespace internal {
+
+template <typename Message>
+static Status CheckMessagesEquivalent(const Buffer& l_buf, const Buffer& r_buf) {
+  ARROW_ASSIGN_OR_RAISE(auto l, ParseFromBuffer<Message>(l_buf));
+  ARROW_ASSIGN_OR_RAISE(auto r, ParseFromBuffer<Message>(r_buf));
+
+  using google::protobuf::util::MessageDifferencer;
+
+  std::string out;
+  google::protobuf::io::StringOutputStream out_stream{&out};
+  MessageDifferencer::StreamReporter reporter{&out_stream};
+
+  MessageDifferencer differencer;
+  differencer.set_message_field_comparison(MessageDifferencer::EQUIVALENT);
+  differencer.ReportDifferencesTo(&reporter);
+
+  if (differencer.Compare(l, r)) {
+    return Status::OK();
+  }
+  return Status::Invalid("Messages were not equivalent: ", out);
+}
+
+Status CheckMessagesEquivalent(util::string_view message_name, const Buffer& l_buf,
+                               const Buffer& r_buf) {
+  if (message_name == "Type") {
+    return CheckMessagesEquivalent<substrait::Type>(l_buf, r_buf);
+  }
+
+  if (message_name == "NamedStruct") {
+    return CheckMessagesEquivalent<substrait::NamedStruct>(l_buf, r_buf);
+  }
+
+  if (message_name == "Schema") {
+    return Status::Invalid(
+        "There is no substrait message named Schema. The substrait message type which "
+        "corresponds to Schemas is NamedStruct");
+  }

Review comment:
       Is this a testing thing?  Who exactly are these error messages intended for?

##########
File path: cpp/src/arrow/engine/substrait/extension_set.cc
##########
@@ -0,0 +1,346 @@
+// 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/engine/substrait/extension_set.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/util/hash_util.h"
+#include "arrow/util/hashing.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+namespace {
+
+struct TypePtrHashEq {
+  template <typename Ptr>
+  size_t operator()(const Ptr& type) const {
+    return type->Hash();
+  }
+
+  template <typename Ptr>
+  bool operator()(const Ptr& l, const Ptr& r) const {
+    return *l == *r;
+  }
+};
+
+struct IdHashEq {
+  using Id = ExtensionSet::Id;
+
+  size_t operator()(Id id) const {
+    constexpr ::arrow::internal::StringViewHash hash = {};
+    auto out = static_cast<size_t>(hash(id.uri));
+    ::arrow::internal::hash_combine(out, hash(id.name));
+    return out;
+  }
+
+  bool operator()(Id l, Id r) const { return l.uri == r.uri && l.name == r.name; }
+};
+
+}  // namespace
+
+struct ExtensionSet::Impl {
+  void AddUri(util::string_view uri, ExtensionSet* self) {
+    if (uris_.find(uri) != uris_.end()) return;
+
+    self->uris_.push_back(uri);
+    uris_.insert(self->uris_.back());  // lookup helper's keys should reference memory
+                                       // owned by this ExtensionSet

Review comment:
       It's not really clear what "lookup helper" is

##########
File path: cpp/src/arrow/engine/substrait/extension_set.cc
##########
@@ -0,0 +1,346 @@
+// 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/engine/substrait/extension_set.h"
+
+#include <unordered_map>
+#include <unordered_set>
+
+#include "arrow/util/hash_util.h"
+#include "arrow/util/hashing.h"
+#include "arrow/util/string_view.h"
+
+namespace arrow {
+namespace engine {
+namespace {
+
+struct TypePtrHashEq {
+  template <typename Ptr>
+  size_t operator()(const Ptr& type) const {
+    return type->Hash();
+  }
+
+  template <typename Ptr>
+  bool operator()(const Ptr& l, const Ptr& r) const {
+    return *l == *r;
+  }
+};
+
+struct IdHashEq {
+  using Id = ExtensionSet::Id;
+
+  size_t operator()(Id id) const {
+    constexpr ::arrow::internal::StringViewHash hash = {};
+    auto out = static_cast<size_t>(hash(id.uri));
+    ::arrow::internal::hash_combine(out, hash(id.name));
+    return out;
+  }
+
+  bool operator()(Id l, Id r) const { return l.uri == r.uri && l.name == r.name; }
+};
+
+}  // namespace
+
+struct ExtensionSet::Impl {
+  void AddUri(util::string_view uri, ExtensionSet* self) {
+    if (uris_.find(uri) != uris_.end()) return;
+
+    self->uris_.push_back(uri);
+    uris_.insert(self->uris_.back());  // lookup helper's keys should reference memory
+                                       // owned by this ExtensionSet
+  }
+
+  Status CheckHasUri(util::string_view uri) {
+    if (uris_.find(uri) != uris_.end()) return Status::OK();
+
+    return Status::Invalid(
+        "Uri ", uri,
+        " was referenced by an extension but was not declared in the ExtensionSet.");
+  }
+
+  uint32_t EncodeType(Id id, const std::shared_ptr<DataType>& type, bool is_variation,
+                      ExtensionSet* self) {
+    // note: at this point we're guaranteed to have an Id which points to memory owned by
+    // the set's registry.
+    AddUri(id.uri, self);
+    auto it_success = types_.emplace(id, static_cast<uint32_t>(types_.size()));
+
+    if (it_success.second) {
+      DCHECK_EQ(self->type_ids_.size(), self->types_.size());
+      self->type_ids_.push_back(id);
+      self->types_.push_back(type);
+      self->type_is_variation_.push_back(is_variation);
+    }
+
+    return it_success.first->second;
+  }
+
+  uint32_t EncodeFunction(Id id, util::string_view function_name, ExtensionSet* self) {
+    // note: at this point we're guaranteed to have an Id which points to memory owned by
+    // the set's registry.
+    AddUri(id.uri, self);
+    auto it_success = functions_.emplace(id, static_cast<uint32_t>(functions_.size()));
+
+    if (it_success.second) {
+      DCHECK_EQ(self->function_ids_.size(), self->function_names_.size());
+      self->function_ids_.push_back(id);
+      self->function_names_.push_back(function_name);
+    }
+
+    return it_success.first->second;
+  }
+
+  std::unordered_set<util::string_view, ::arrow::internal::StringViewHash> uris_;
+  std::unordered_map<Id, uint32_t, IdHashEq, IdHashEq> types_, functions_;
+};
+
+ExtensionSet::ExtensionSet(ExtensionIdRegistry* registry)
+    : registry_(registry), impl_(new Impl(), [](Impl* impl) { delete impl; }) {}
+
+Result<ExtensionSet> ExtensionSet::Make(std::vector<util::string_view> uris,
+                                        std::vector<Id> type_ids,
+                                        std::vector<bool> type_is_variation,
+                                        std::vector<Id> function_ids,
+                                        ExtensionIdRegistry* registry) {
+  ExtensionSet set;
+  set.registry_ = registry;
+
+  // TODO(bkietz) move this into the registry as registry->OwnUris(&uris) or so
+  std::unordered_set<util::string_view, ::arrow::internal::StringViewHash>
+      uris_owned_by_registry;
+  for (util::string_view uri : registry->Uris()) {
+    uris_owned_by_registry.insert(uri);
+  }
+
+  for (auto& uri : uris) {
+    if (uri.empty()) continue;
+    auto it = uris_owned_by_registry.find(uri);
+    if (it == uris_owned_by_registry.end()) {
+      return Status::KeyError("Uri '", uri, "' not found in registry");
+    }
+    uri = *it;  // Ensure uris point into the registry's memory
+    set.impl_->AddUri(*it, &set);
+  }
+
+  if (type_ids.size() != type_is_variation.size()) {
+    return Status::Invalid("Received ", type_ids.size(), " type ids but a ",
+                           type_is_variation.size(), "-long is_variation vector");
+  }
+
+  set.types_.resize(type_ids.size());
+
+  for (size_t i = 0; i < type_ids.size(); ++i) {
+    if (type_ids[i].empty()) continue;
+    RETURN_NOT_OK(set.impl_->CheckHasUri(type_ids[i].uri));
+
+    if (auto rec = registry->GetType(type_ids[i], type_is_variation[i])) {
+      set.types_[i] = rec->type;
+      type_ids[i] = rec->id;  // use Id which references memory owned by the registry
+      continue;
+    }
+    return Status::Invalid("Type", (type_is_variation[i] ? " variation" : ""), " ",
+                           type_ids[i].uri, "#", type_ids[i].name, " not found");
+  }
+
+  set.function_names_.resize(function_ids.size());
+
+  for (size_t i = 0; i < function_ids.size(); ++i) {
+    if (function_ids[i].empty()) continue;
+    RETURN_NOT_OK(set.impl_->CheckHasUri(function_ids[i].uri));
+
+    if (auto rec = registry->GetFunction(function_ids[i])) {
+      set.function_names_[i] = rec->function_name;
+      function_ids[i] = rec->id;  // use Id which references memory owned by the registry
+      continue;
+    }
+    return Status::Invalid("Function ", function_ids[i].uri, "#", type_ids[i].name,
+                           " not found");
+  }
+
+  set.function_ids_ = std::move(function_ids);
+  set.type_ids_ = std::move(type_ids);
+  set.type_is_variation_ = std::move(type_is_variation);
+  set.uris_ = std::move(uris);
+
+  return std::move(set);
+}
+
+Result<uint32_t> ExtensionSet::EncodeType(const DataType& type) {
+  if (auto rec = registry_->GetType(type)) {
+    return impl_->EncodeType(rec->id, rec->type, rec->is_variation, this);
+  }
+  return Status::KeyError("type ", type.ToString(), " not found in the registry");
+}
+
+Result<uint32_t> ExtensionSet::EncodeFunction(util::string_view function_name) {
+  if (auto rec = registry_->GetFunction(function_name)) {
+    return impl_->EncodeFunction(rec->id, rec->function_name, this);
+  }
+  return Status::KeyError("function ", function_name, " not found in the registry");
+}
+
+template <typename KeyToIndex, typename Key>
+const int* GetIndex(const KeyToIndex& key_to_index, const Key& key) {
+  auto it = key_to_index.find(key);
+  if (it == key_to_index.end()) return nullptr;
+  return &it->second;
+}
+
+ExtensionIdRegistry* default_extension_id_registry() {
+  static struct Impl : ExtensionIdRegistry {
+    Impl() {
+      struct TypeName {
+        std::shared_ptr<DataType> type;
+        util::string_view name;
+      };
+
+      for (TypeName e : {
+               TypeName{uint8(), "u8"},
+               TypeName{uint16(), "u16"},
+               TypeName{uint32(), "u32"},
+               TypeName{uint64(), "u64"},
+           }) {
+        DCHECK_OK(RegisterType({kArrowExtTypesUri, e.name}, std::move(e.type),
+                               /*is_variation=*/true));
+      }
+
+      for (TypeName e : {
+               TypeName{null(), "null"},
+           }) {
+        DCHECK_OK(RegisterType({kArrowExtTypesUri, e.name}, std::move(e.type),
+                               /*is_variation=*/false));
+      }
+
+      for (util::string_view name : {
+               "add",
+           }) {
+        DCHECK_OK(RegisterFunction({kArrowExtTypesUri, name}, name.to_string()));
+      }
+    }
+
+    std::vector<util::string_view> Uris() const override {
+      return {uris_.begin(), uris_.end()};
+    }
+
+    util::optional<TypeRecord> GetType(const DataType& type) const override {
+      if (auto index = GetIndex(type_to_index_, &type)) {
+        return TypeRecord{type_ids_[*index], types_[*index], type_is_variation_[*index]};
+      }
+      return {};
+    }
+
+    util::optional<TypeRecord> GetType(Id id, bool is_variation) const override {
+      if (auto index =
+              GetIndex(is_variation ? variation_id_to_index_ : id_to_index_, id)) {
+        return TypeRecord{type_ids_[*index], types_[*index], type_is_variation_[*index]};
+      }
+      return {};
+    }
+
+    Status RegisterType(Id id, std::shared_ptr<DataType> type,
+                        bool is_variation) override {
+      DCHECK_EQ(type_ids_.size(), types_.size());
+      DCHECK_EQ(type_ids_.size(), type_is_variation_.size());
+
+      Id copied_id{*uris_.emplace(id.uri.to_string()).first,
+                   *names_.emplace(id.name.to_string()).first};
+
+      auto index = static_cast<int>(type_ids_.size());
+
+      auto* id_to_index = is_variation ? &variation_id_to_index_ : &id_to_index_;
+      auto it_success = id_to_index->emplace(copied_id, index);
+
+      if (!it_success.second) {
+        return Status::Invalid("Type id was already registered");
+      }
+
+      if (!type_to_index_.emplace(type.get(), index).second) {
+        id_to_index->erase(it_success.first);
+        return Status::Invalid("Type was already registered");
+      }
+
+      type_ids_.push_back(copied_id);
+      types_.push_back(std::move(type));
+      type_is_variation_.push_back(is_variation);
+      return Status::OK();
+    }
+
+    util::optional<FunctionRecord> GetFunction(
+        util::string_view arrow_function_name) const override {
+      if (auto index = GetIndex(function_name_to_index_, arrow_function_name)) {
+        return FunctionRecord{function_ids_[*index], *function_name_ptrs_[*index]};
+      }
+      return {};
+    }
+
+    util::optional<FunctionRecord> GetFunction(Id id) const override {
+      if (auto index = GetIndex(function_id_to_index_, id)) {
+        return FunctionRecord{function_ids_[*index], *function_name_ptrs_[*index]};
+      }
+      return {};
+    }
+
+    Status RegisterFunction(Id id, std::string arrow_function_name) override {
+      DCHECK_EQ(function_ids_.size(), function_name_ptrs_.size());
+
+      Id copied_id{*uris_.emplace(id.uri.to_string()).first,
+                   *names_.emplace(id.name.to_string()).first};
+
+      const std::string& copied_function_name{
+          *function_names_.emplace(std::move(arrow_function_name)).first};
+
+      auto index = static_cast<int>(function_ids_.size());
+
+      auto it_success = function_id_to_index_.emplace(copied_id, index);
+
+      if (!it_success.second) {
+        return Status::Invalid("Function id was already registered");
+      }
+
+      if (!function_name_to_index_.emplace(copied_function_name, index).second) {
+        function_id_to_index_.erase(it_success.first);
+        return Status::Invalid("Function name was already registered");
+      }
+
+      function_name_ptrs_.push_back(&copied_function_name);
+      function_ids_.push_back(copied_id);
+      return Status::OK();
+    }
+
+    // owning storage of uris, names, (arrow::)function_names, types
+    //    note that storing strings like this is safe since references into an
+    //    unordered_set are not invalidated on insertion
+    std::unordered_set<std::string> uris_, names_, function_names_;

Review comment:
       You're introducing quite a bit of complexity simply to avoid `unordered_map` lookups during plan building.  Off the cuff it feels like a premature optimization but I suppose it can't hurt as long as the anchors are low numbers.

##########
File path: cpp/src/arrow/engine/substrait/plan_internal.h
##########
@@ -0,0 +1,39 @@
+// 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 "arrow/engine/substrait/extension_set.h"
+#include "arrow/engine/visibility.h"
+#include "arrow/type_fwd.h"
+
+#include "substrait/plan.pb.h"  // IWYU pragma: export
+
+namespace arrow {
+namespace engine {
+
+ARROW_ENGINE_EXPORT
+Status AddExtensionSetToPlan(const ExtensionSet&, substrait::Plan*);

Review comment:
       At the moment this is completely unused but I can see how it would be useful if we are going to be creating plans.  But I don't really know if we're going to be doing that.  What do you see as the future for this?
   
   Either way some docstrings about what this function does could be useful.  It could be something brief like "Adds extension definitions to a Substrait plan to be referenced by anchors throughout the rest of the plan"

##########
File path: cpp/src/arrow/engine/substrait/serde.cc
##########
@@ -0,0 +1,232 @@
+// 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/engine/substrait/serde.h"
+
+#include "arrow/engine/substrait/expression_internal.h"
+#include "arrow/engine/substrait/plan_internal.h"
+#include "arrow/engine/substrait/relation_internal.h"
+#include "arrow/engine/substrait/type_internal.h"
+#include "arrow/util/string_view.h"
+
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/io/zero_copy_stream_impl_lite.h>
+#include <google/protobuf/message.h>
+#include <google/protobuf/util/json_util.h>
+#include <google/protobuf/util/message_differencer.h>
+#include <google/protobuf/util/type_resolver_util.h>
+
+namespace arrow {
+namespace engine {
+
+Status ParseFromBufferImpl(const Buffer& buf, const std::string& full_name,
+                           google::protobuf::Message* message) {
+  google::protobuf::io::ArrayInputStream buf_stream{buf.data(),
+                                                    static_cast<int>(buf.size())};
+
+  if (message->ParseFromZeroCopyStream(&buf_stream)) {
+    return Status::OK();
+  }
+  return Status::IOError("ParseFromZeroCopyStream failed for ", full_name);
+}
+
+template <typename Message>
+Result<Message> ParseFromBuffer(const Buffer& buf) {
+  Message message;
+  ARROW_RETURN_NOT_OK(
+      ParseFromBufferImpl(buf, Message::descriptor()->full_name(), &message));
+  return message;
+}
+
+Result<compute::Declaration> DeserializeRelation(const Buffer& buf,
+                                                 const ExtensionSet& ext_set) {
+  ARROW_ASSIGN_OR_RAISE(auto rel, ParseFromBuffer<substrait::Rel>(buf));
+  return FromProto(rel, ext_set);
+}
+
+Result<std::vector<compute::Declaration>> DeserializePlan(
+    const Buffer& buf, const ConsumerFactory& consumer_factory,
+    ExtensionSet* ext_set_out) {
+  ARROW_ASSIGN_OR_RAISE(auto plan, ParseFromBuffer<substrait::Plan>(buf));
+
+  ARROW_ASSIGN_OR_RAISE(auto ext_set, GetExtensionSetFromPlan(plan));
+
+  std::vector<compute::Declaration> sink_decls;
+  for (const substrait::PlanRel& plan_rel : plan.relations()) {
+    if (plan_rel.has_root()) {
+      return Status::NotImplemented("substrait::PlanRel with custom output field names");
+    }

Review comment:
       Is this just a TODO or is there some reason we can't handle this with a final `project`?  If it's just a TODO let's add a JIRA.




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

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

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