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 2021/08/26 23:10:14 UTC

[GitHub] [arrow] jacques-n commented on a change in pull request #10934: [RFC] Arrow Compute Serialized Intermediate Representation draft for discussion

jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697019516



##########
File path: format/experimental/computeir/Plan.fbs
##########
@@ -0,0 +1,33 @@
+// 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 "Relation.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A specification of a query.
+table Plan {
+  /// One or more output relations.
+  sinks: [Relation] (required);
+
+  /// If this Plan was derived from another (for example by running
+  /// an optimization pass), that plan may be included here to
+  /// provide a backtrace of derivations.
+  derived_from: Plan;

Review comment:
       This feels premature to me. Immediately I question why it isn't an array (since a plan may go through multiple transformations). Can we remove until we have a strong need for it?

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// Output remapping of ordinals for a given operation
+  output_mapping: Emit (required);
+
+  /// Arguments for custom operations
+  options: InlineBuffer;
+}
+
+/// Filter operation
+table Filter {
+  base: RelBase (required);
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  predicate: Expression (required);
+}
+
+/// Projection
+table Project {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// Aggregate operation
+table Aggregate {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the aggregate relation's output.
+  aggregations: [Expression] (required);
+  /// Keys by which `aggregations` will be grouped.
+  keys: [Expression] (required);
+}
+
+enum CanonicalJoinKindId : uint8 {
+  Anti,
+  Cross,
+  FullOuter,
+  Inner,
+  LeftOuter,
+  LeftSemi,
+  RightOuter,
+  // TODO: Window
+  // TODO: AsOf

Review comment:
       Not convinced that window and asof should exist here. I'm also somewhat torn on combining traditional joins and correlations in Calcite speak. They feel different enough to separate out.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {
+  /// Expression from which to extract components.
+  expression: Expression (required);
+
+  /// Field to extract from `expression`.
+  field: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values.
+enum Ordering : uint8 {
+  ASCENDING_THEN_NULLS,
+  DESCENDING_THEN_NULLS,
+  NULLS_THEN_ASCENDING,
+  NULLS_THEN_DESCENDING
+}
+
+/// An expression with an order
+table SortKey {
+  expression: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// Boundary is unbounded
+table Unbounded {}
+
+union ConcreteBoundImpl {
+  Expression,
+  Unbounded,
+}
+
+/// Boundary is preceding rows, determined by the contained expression
+table Preceding {
+  ipml: ConcreteBoundImpl (required);
+}
+
+/// Boundary is following rows, determined by the contained expression
+table Following {
+  impl: ConcreteBoundImpl (required);
+}
+
+/// Boundary is the current row
+table CurrentRow {}
+
+union BoundImpl {
+  Preceding,
+  Following,
+  CurrentRow,
+}
+
+/// Boundary of a window
+table Bound {
+  impl: BoundImpl (required);
+}
+
+/// The kind of window function to be executed.
+enum Frame : uint8 {
+  Rows,
+  Range,
+}
+
+/// An expression representing a window function call.
+table WindowCall {
+  /// The kind of window frame
+  kind: Frame;
+  /// The expression to operate over
+  expression: Expression (required);
+  /// Partition keys
+  partitions: [Expression] (required);
+  /// Sort keys
+  orderings: [SortKey] (required);
+  /// Lower window bound
+  lower_bound: Bound (required);
+  /// Upper window bound
+  upper_bound: Bound (required);
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalAggregateId : uint32 {
+  All,
+  Any,
+  Count,
+  CountTable,
+  Mean,
+  Min,
+  Max,
+  Product,
+  Sum,
+  Variance,
+  StandardDev,
+}
+
+
+table CanonicalAggregate {
+  id: CanonicalAggregateId;
+}
+
+table NonCanonicalAggregate {
+  name_space: string;
+  name: string (required);
+}
+
+union AggregateImpl {
+  CanonicalAggregate,
+  NonCanonicalAggregate,
+}
+
+table AggregateCall {
+  /// The kind of aggregate function being executed
+  kind: AggregateImpl (required);
+
+  /// Aggregate expression arguments
+  arguments: [Expression] (required);

Review comment:
       Is array required allow an empty list? Just want to clarify since things like COUNT() don't require any arguments.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {
+  /// Expression from which to extract components.
+  expression: Expression (required);
+
+  /// Field to extract from `expression`.
+  field: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values.
+enum Ordering : uint8 {
+  ASCENDING_THEN_NULLS,
+  DESCENDING_THEN_NULLS,
+  NULLS_THEN_ASCENDING,
+  NULLS_THEN_DESCENDING
+}
+
+/// An expression with an order
+table SortKey {
+  expression: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// Boundary is unbounded
+table Unbounded {}
+
+union ConcreteBoundImpl {
+  Expression,
+  Unbounded,
+}
+
+/// Boundary is preceding rows, determined by the contained expression
+table Preceding {
+  ipml: ConcreteBoundImpl (required);
+}
+
+/// Boundary is following rows, determined by the contained expression
+table Following {
+  impl: ConcreteBoundImpl (required);
+}
+
+/// Boundary is the current row
+table CurrentRow {}
+
+union BoundImpl {
+  Preceding,
+  Following,
+  CurrentRow,
+}
+
+/// Boundary of a window
+table Bound {
+  impl: BoundImpl (required);
+}
+
+/// The kind of window function to be executed.
+enum Frame : uint8 {
+  Rows,
+  Range,
+}
+
+/// An expression representing a window function call.
+table WindowCall {
+  /// The kind of window frame
+  kind: Frame;
+  /// The expression to operate over
+  expression: Expression (required);
+  /// Partition keys
+  partitions: [Expression] (required);
+  /// Sort keys
+  orderings: [SortKey] (required);
+  /// Lower window bound
+  lower_bound: Bound (required);
+  /// Upper window bound
+  upper_bound: Bound (required);
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalAggregateId : uint32 {
+  All,
+  Any,
+  Count,
+  CountTable,
+  Mean,
+  Min,
+  Max,
+  Product,
+  Sum,
+  Variance,
+  StandardDev,
+}
+
+
+table CanonicalAggregate {
+  id: CanonicalAggregateId;
+}
+
+table NonCanonicalAggregate {
+  name_space: string;
+  name: string (required);
+}
+
+union AggregateImpl {
+  CanonicalAggregate,
+  NonCanonicalAggregate,
+}
+
+table AggregateCall {
+  /// The kind of aggregate function being executed
+  kind: AggregateImpl (required);
+
+  /// Aggregate expression arguments
+  arguments: [Expression] (required);
+
+  /// Possible ordering.
+  orderings: [SortKey];
+
+  /// optional per-aggregate filtering
+  predicate: Expression;
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a field from a Relation
+/// - a call to a named function
+/// - a case expression
+/// - a cast expression
+/// - an extract operation
+/// - a window function call
+/// - an aggregate function call
+///
+/// The expressions here that look like function calls such as
+/// Cast,Case and Extract are special in that while they might
+/// fit into a Call, they don't cleanly do so without having
+/// to pass around non-expression arguments as metadata.
+///
+/// AggregateCall and WindowCall are also separate variants
+/// due to special options for each that don't apply to generic
+/// function calls. Again this is done to make it easier
+/// for consumers to deal with the structure of the operation
+union ExpressionImpl {
+  Literal,
+  FieldRef,
+  Call,
+  Case,
+  Cast,
+  Extract,
+  WindowCall,
+  AggregateCall,
+}
+
+/// Expression types
+///
+/// Expressions have a concrete `impl` value, which is a specific operation
+/// They also have a `type` field, which is the output type of the expression,
+/// regardless of operation type.
+///
+/// The only exception so far is Cast, which has a type as input argument, which
+/// is equal to output type.
+table Expression {
+  impl: ExpressionImpl (required);
+
+  /// The type of the expression.
+  ///
+  /// This is a field, because the Type union in Schema.fbs
+  /// isn't self-contained: Fields are necessary to describe complex types
+  /// and there's currently no reason to optimize the storage of this.
+  type: org.apache.arrow.flatbuf.Field;

Review comment:
       I don't understand why we would have an output type for each operation. This is duplicative and should be easily derivable.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {

Review comment:
       I don't understand what this is. Especially the field:string part.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {

Review comment:
       I don't think specific functions should be part of the IR. A couple of reasons:
   
   - Each of these will actually have many variations and each should be identified separately add(int,int) => int vs add(bigint,bigint) => bigint.
   - The speed at which functions are created/added should happen at a much more rapid pace than changes to the core IR.
   - Just because new functions are introduced doesn't mean we should change the format version number. The absence of a function in a particular consumer shouldn't really matter.
   

##########
File path: format/experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,177 @@
+// 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.
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+table ArrayLiteral {
+  values: [Literal];
+}
+
+table StructLiteral {
+  values: [KeyValue];
+}
+
+table KeyValue {
+  key: string;
+  value: Literal;
+}
+
+table MapLiteral {
+  values: [KeyValue];
+}
+
+table Int8Literal {
+  value: int8;
+}
+
+table Int16Literal {
+  value: int16;
+}
+
+table Int32Literal {
+  value: int32;
+}
+
+table Int64Literal {
+  value: int64;
+}
+
+table UInt8Literal {
+  value: uint8;
+}
+
+table UInt16Literal {
+  value: uint16;
+}
+
+table UInt32Literal {
+  value: uint32;
+}
+
+table UInt64Literal {
+  value: uint64;
+}
+
+table Float16Literal {
+  value: uint16;
+}
+
+table Float32Literal {
+  value: float32;
+}
+
+table Float64Literal {
+  value: float64;
+}
+
+table DecimalLiteral {
+  value: [uint8]; // 128 bit value.
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+table DateLiteral {
+  // milliseconds since epoch
+  value: int64;
+}
+
+table TimeLiteral {
+  // nanosecond time value.
+  value: int64;
+}
+
+table TimestampLiteral {
+  // microsecond timestamp.
+  value: int64;
+
+  // timezone value, same definition as used in Schema.fbs.
+  timezone: string;
+}
+
+table IntervalLiteralMonths {
+  months: int32;
+}
+
+table IntervalLiteralDaysMilliseconds {
+  days: int32;
+  milliseconds: int32;
+}
+
+union IntervalLiteralImpl {
+  IntervalLiteralMonths,
+  IntervalLiteralDaysMilliseconds,
+}
+
+table IntervalLiteral {
+  value: IntervalLiteralImpl;
+}
+
+table BinaryLiteral {
+  value: [byte];
+}
+
+table StringLiteral {
+  value: string;
+}
+
+// no union literal is defined as only one branch of a union can be resolved.
+// no literals for large string/binary types as flatbuffer is limited to 2gb.
+
+union LiteralImpl {
+  NullLiteral,
+  BooleanLiteral,
+
+  Int8Literal,
+  Int16Literal,
+  Int32Literal,
+  Int64Literal,
+
+  UInt8Literal,
+  UInt16Literal,
+  UInt32Literal,
+  UInt64Literal,
+
+  DateLiteral,
+  TimeLiteral,
+  TimestampLiteral,
+  IntervalLiteral,
+
+  DecimalLiteral,
+
+  Float16Literal,
+  Float32Literal,
+  Float64Literal,
+
+  ArrayLiteral,
+  StructLiteral,
+  MapLiteral,
+
+  StringLiteral,
+  BinaryLiteral,
+}
+
+table Literal {

Review comment:
       Does this intermediate table just exist to avoid resolution ordering issues in flatbuf? Why not just call the LiteralImpl union Literal? Is it so we can use root_type? It feels like it just bloats the definition and the serialization, no?

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// Output remapping of ordinals for a given operation
+  output_mapping: Emit (required);
+
+  /// Arguments for custom operations
+  options: InlineBuffer;
+}
+
+/// Filter operation
+table Filter {
+  base: RelBase (required);
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  predicate: Expression (required);
+}
+
+/// Projection
+table Project {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// Aggregate operation
+table Aggregate {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the aggregate relation's output.
+  aggregations: [Expression] (required);
+  /// Keys by which `aggregations` will be grouped.
+  keys: [Expression] (required);
+}
+
+enum CanonicalJoinKindId : uint8 {
+  Anti,
+  Cross,
+  FullOuter,
+  Inner,
+  LeftOuter,
+  LeftSemi,
+  RightOuter,
+  // TODO: Window
+  // TODO: AsOf
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string;
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind,
+  NonCanonicalJoinKind,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table Join {
+  base: RelBase (required);
+  /// The expression which will be evaluated against rows from each
+  /// input to determine whether they should be included in the
+  /// join relation's output.
+  on_expression: Expression (required);
+  /// The kind of join to use.
+  join_kind: JoinKind (required);
+}
+
+/// Order by relation
+table OrderBy {
+  base: RelBase (required);
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// Limit operation
+table Limit {
+  base: RelBase (required);
+  /// The maximum number of rows of output.
+  count: long;
+}
+
+/// Common table expresssion
+table Common {
+  base: RelBase (required);
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string (required);
+}
+
+/// The kind of set operation being performed.
+enum CanonicalSetOpKindId : uint8 {
+  Union,
+  Intersection,
+  Difference,
+}
+
+table CanonicalSetOpKind {
+  id: CanonicalSetOpKindId;
+}
+
+table NonCanonicalSetOpKind {
+  name_space: string;
+  name: string (required);
+}
+
+/// The kind of set operation
+union SetOpKind {
+  CanonicalSetOpKind,
+  NonCanonicalSetOpKind,
+}
+
+/// A set operation on two or more relations
+table SetOperation {
+  base: RelBase (required);
+  /// The kind of set operation
+  set_op: SetOpKind (required);
+}
+
+/// Literal relation
+table LiteralRelation {
+  base: RelBase (required);
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// A user-defined relation.
+table Custom {
+  base: RelBase (required);
+  /// NB: the schema is optional here, since a custom relation
+  /// may or may not need it.
+  schema: org.apache.arrow.flatbuf.Schema;
+}
+
+/// A table read
+table Read {
+  base: RelBase (required);
+  resource: string (required);
+  schema: org.apache.arrow.flatbuf.Schema (required);
+}
+
+/// A table write
+table Write {

Review comment:
       I think Read & Write should be removed from this patch. It seems like we need to spend a decent amount of time deciding what kind of structure could be there. A required string (and string parsing?) seems like the wrong place to start.

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// Output remapping of ordinals for a given operation
+  output_mapping: Emit (required);
+
+  /// Arguments for custom operations
+  options: InlineBuffer;
+}
+
+/// Filter operation
+table Filter {
+  base: RelBase (required);
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  predicate: Expression (required);
+}
+
+/// Projection
+table Project {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// Aggregate operation
+table Aggregate {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the aggregate relation's output.
+  aggregations: [Expression] (required);
+  /// Keys by which `aggregations` will be grouped.
+  keys: [Expression] (required);
+}
+
+enum CanonicalJoinKindId : uint8 {
+  Anti,
+  Cross,
+  FullOuter,
+  Inner,
+  LeftOuter,
+  LeftSemi,
+  RightOuter,
+  // TODO: Window
+  // TODO: AsOf
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string;
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind,
+  NonCanonicalJoinKind,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table Join {
+  base: RelBase (required);
+  /// The expression which will be evaluated against rows from each
+  /// input to determine whether they should be included in the
+  /// join relation's output.
+  on_expression: Expression (required);
+  /// The kind of join to use.
+  join_kind: JoinKind (required);
+}
+
+/// Order by relation
+table OrderBy {
+  base: RelBase (required);
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// Limit operation
+table Limit {
+  base: RelBase (required);
+  /// The maximum number of rows of output.
+  count: long;
+}
+
+/// Common table expresssion
+table Common {
+  base: RelBase (required);
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string (required);
+}
+
+/// The kind of set operation being performed.
+enum CanonicalSetOpKindId : uint8 {
+  Union,
+  Intersection,
+  Difference,
+}
+
+table CanonicalSetOpKind {
+  id: CanonicalSetOpKindId;
+}
+
+table NonCanonicalSetOpKind {
+  name_space: string;
+  name: string (required);
+}
+
+/// The kind of set operation
+union SetOpKind {
+  CanonicalSetOpKind,
+  NonCanonicalSetOpKind,
+}
+
+/// A set operation on two or more relations
+table SetOperation {
+  base: RelBase (required);
+  /// The kind of set operation
+  set_op: SetOpKind (required);
+}
+
+/// Literal relation
+table LiteralRelation {
+  base: RelBase (required);
+  /// The columns of this literal relation.
+  columns: [Literal] (required);

Review comment:
       I think this should be [[Literal]] since a literal relation could be many records.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {
+  /// Expression from which to extract components.
+  expression: Expression (required);
+
+  /// Field to extract from `expression`.
+  field: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values.
+enum Ordering : uint8 {
+  ASCENDING_THEN_NULLS,
+  DESCENDING_THEN_NULLS,
+  NULLS_THEN_ASCENDING,
+  NULLS_THEN_DESCENDING

Review comment:
       We should also have Clustered here. Clustered means collated but unordered. For example:
   if you had a,b,c,a,b,c a clustered order might be b,b,a,a,c,c. Clustered is useful for operations like streaming aggregations or writing partitioned data. It doesn't need to be sorted, just requires ensuring the stream has all of the same value together.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {

Review comment:
       My suggestion is ids here, a separate yaml or similar structured doc that lists canonical functions that includes not only structured data around type, etc but also description/details around specific behavior.

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// Output remapping of ordinals for a given operation
+  output_mapping: Emit (required);
+
+  /// Arguments for custom operations
+  options: InlineBuffer;
+}
+
+/// Filter operation
+table Filter {
+  base: RelBase (required);
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  predicate: Expression (required);
+}
+
+/// Projection
+table Project {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// Aggregate operation
+table Aggregate {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the aggregate relation's output.
+  aggregations: [Expression] (required);
+  /// Keys by which `aggregations` will be grouped.
+  keys: [Expression] (required);

Review comment:
       This should actually support the grouping set concept. See my patch. Logically [[Expression]] instead of [Expression], basically. The absence of grouping sets early in Calcite made the later addition a mess in many ways.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {
+  /// Expression from which to extract components.
+  expression: Expression (required);
+
+  /// Field to extract from `expression`.
+  field: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values.
+enum Ordering : uint8 {
+  ASCENDING_THEN_NULLS,
+  DESCENDING_THEN_NULLS,
+  NULLS_THEN_ASCENDING,
+  NULLS_THEN_DESCENDING
+}
+
+/// An expression with an order
+table SortKey {
+  expression: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// Boundary is unbounded
+table Unbounded {}
+
+union ConcreteBoundImpl {
+  Expression,
+  Unbounded,
+}
+
+/// Boundary is preceding rows, determined by the contained expression
+table Preceding {
+  ipml: ConcreteBoundImpl (required);
+}
+
+/// Boundary is following rows, determined by the contained expression
+table Following {
+  impl: ConcreteBoundImpl (required);

Review comment:
       in general, your use of Impl throughout the code is confusing. Since all of these things are serialized forms, not implementations of the operations. And in most cases, the children are impls, the wrapper is a wrapper, not an impl.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {
+  /// Expression from which to extract components.
+  expression: Expression (required);
+
+  /// Field to extract from `expression`.
+  field: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values.
+enum Ordering : uint8 {
+  ASCENDING_THEN_NULLS,
+  DESCENDING_THEN_NULLS,
+  NULLS_THEN_ASCENDING,
+  NULLS_THEN_DESCENDING
+}
+
+/// An expression with an order
+table SortKey {
+  expression: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// Boundary is unbounded
+table Unbounded {}
+
+union ConcreteBoundImpl {
+  Expression,
+  Unbounded,
+}
+
+/// Boundary is preceding rows, determined by the contained expression
+table Preceding {
+  ipml: ConcreteBoundImpl (required);
+}
+
+/// Boundary is following rows, determined by the contained expression
+table Following {
+  impl: ConcreteBoundImpl (required);
+}
+
+/// Boundary is the current row
+table CurrentRow {}
+
+union BoundImpl {
+  Preceding,
+  Following,
+  CurrentRow,
+}
+
+/// Boundary of a window
+table Bound {
+  impl: BoundImpl (required);
+}
+
+/// The kind of window function to be executed.
+enum Frame : uint8 {
+  Rows,
+  Range,
+}
+
+/// An expression representing a window function call.
+table WindowCall {
+  /// The kind of window frame
+  kind: Frame;
+  /// The expression to operate over
+  expression: Expression (required);
+  /// Partition keys
+  partitions: [Expression] (required);
+  /// Sort keys
+  orderings: [SortKey] (required);
+  /// Lower window bound
+  lower_bound: Bound (required);
+  /// Upper window bound
+  upper_bound: Bound (required);
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalAggregateId : uint32 {
+  All,

Review comment:
       Comment on functions here is same as scalar functions. Don't think they should be in the fbs ir spec.

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// Output remapping of ordinals for a given operation
+  output_mapping: Emit (required);
+
+  /// Arguments for custom operations
+  options: InlineBuffer;
+}
+
+/// Filter operation
+table Filter {
+  base: RelBase (required);
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  predicate: Expression (required);
+}
+
+/// Projection
+table Project {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// Aggregate operation
+table Aggregate {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the aggregate relation's output.
+  aggregations: [Expression] (required);
+  /// Keys by which `aggregations` will be grouped.
+  keys: [Expression] (required);
+}
+
+enum CanonicalJoinKindId : uint8 {
+  Anti,
+  Cross,
+  FullOuter,
+  Inner,
+  LeftOuter,
+  LeftSemi,
+  RightOuter,
+  // TODO: Window
+  // TODO: AsOf
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string;
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind,
+  NonCanonicalJoinKind,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table Join {
+  base: RelBase (required);
+  /// The expression which will be evaluated against rows from each
+  /// input to determine whether they should be included in the
+  /// join relation's output.
+  on_expression: Expression (required);
+  /// The kind of join to use.
+  join_kind: JoinKind (required);
+}
+
+/// Order by relation
+table OrderBy {
+  base: RelBase (required);
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// Limit operation
+table Limit {
+  base: RelBase (required);
+  /// The maximum number of rows of output.
+  count: long;

Review comment:
       this should actually be two numbers. offset and fetch. For example, I want rows 10..100.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {
+  /// Expression from which to extract components.
+  expression: Expression (required);
+
+  /// Field to extract from `expression`.
+  field: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values.
+enum Ordering : uint8 {
+  ASCENDING_THEN_NULLS,
+  DESCENDING_THEN_NULLS,
+  NULLS_THEN_ASCENDING,
+  NULLS_THEN_DESCENDING

Review comment:
       Torn on whether it makes sense to have a set of nulls-not-specified options. Thoughts?

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;

Review comment:
       I'd expect this to be required/default 0.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {
+  /// Expression from which to extract components.
+  expression: Expression (required);
+
+  /// Field to extract from `expression`.
+  field: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values.
+enum Ordering : uint8 {
+  ASCENDING_THEN_NULLS,
+  DESCENDING_THEN_NULLS,
+  NULLS_THEN_ASCENDING,
+  NULLS_THEN_DESCENDING
+}
+
+/// An expression with an order
+table SortKey {
+  expression: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// Boundary is unbounded
+table Unbounded {}
+
+union ConcreteBoundImpl {
+  Expression,
+  Unbounded,
+}
+
+/// Boundary is preceding rows, determined by the contained expression
+table Preceding {
+  ipml: ConcreteBoundImpl (required);
+}
+
+/// Boundary is following rows, determined by the contained expression
+table Following {
+  impl: ConcreteBoundImpl (required);
+}
+
+/// Boundary is the current row
+table CurrentRow {}
+
+union BoundImpl {
+  Preceding,
+  Following,
+  CurrentRow,
+}
+
+/// Boundary of a window
+table Bound {
+  impl: BoundImpl (required);
+}
+
+/// The kind of window function to be executed.
+enum Frame : uint8 {
+  Rows,
+  Range,
+}
+
+/// An expression representing a window function call.
+table WindowCall {
+  /// The kind of window frame
+  kind: Frame;
+  /// The expression to operate over
+  expression: Expression (required);
+  /// Partition keys
+  partitions: [Expression] (required);
+  /// Sort keys
+  orderings: [SortKey] (required);
+  /// Lower window bound
+  lower_bound: Bound (required);
+  /// Upper window bound
+  upper_bound: Bound (required);
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalAggregateId : uint32 {
+  All,
+  Any,
+  Count,
+  CountTable,
+  Mean,
+  Min,
+  Max,
+  Product,
+  Sum,
+  Variance,
+  StandardDev,
+}
+
+
+table CanonicalAggregate {
+  id: CanonicalAggregateId;
+}
+
+table NonCanonicalAggregate {
+  name_space: string;
+  name: string (required);
+}
+
+union AggregateImpl {
+  CanonicalAggregate,
+  NonCanonicalAggregate,
+}
+
+table AggregateCall {
+  /// The kind of aggregate function being executed
+  kind: AggregateImpl (required);
+
+  /// Aggregate expression arguments
+  arguments: [Expression] (required);
+
+  /// Possible ordering.
+  orderings: [SortKey];
+
+  /// optional per-aggregate filtering
+  predicate: Expression;
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a field from a Relation
+/// - a call to a named function
+/// - a case expression
+/// - a cast expression
+/// - an extract operation
+/// - a window function call
+/// - an aggregate function call
+///
+/// The expressions here that look like function calls such as
+/// Cast,Case and Extract are special in that while they might
+/// fit into a Call, they don't cleanly do so without having
+/// to pass around non-expression arguments as metadata.
+///
+/// AggregateCall and WindowCall are also separate variants
+/// due to special options for each that don't apply to generic
+/// function calls. Again this is done to make it easier
+/// for consumers to deal with the structure of the operation
+union ExpressionImpl {
+  Literal,
+  FieldRef,
+  Call,
+  Case,
+  Cast,
+  Extract,
+  WindowCall,
+  AggregateCall,

Review comment:
       I'm not sure that AggregateCall should be an expression. It creates some weirdness since it suggests that you can have sum((sumx)) in the plan. But I don't know what that means. Yes, you could block that on semantic analysis but it feels like the format could just avoid it all together.

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// Output remapping of ordinals for a given operation
+  output_mapping: Emit (required);
+
+  /// Arguments for custom operations
+  options: InlineBuffer;
+}
+
+/// Filter operation
+table Filter {
+  base: RelBase (required);
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  predicate: Expression (required);
+}
+
+/// Projection
+table Project {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// Aggregate operation
+table Aggregate {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the aggregate relation's output.
+  aggregations: [Expression] (required);

Review comment:
       How about calling this measures or calcs or something? As a very, this is not initially obvious that you're talking about the calculations.

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Extract {
+  /// Expression from which to extract components.
+  expression: Expression (required);
+
+  /// Field to extract from `expression`.
+  field: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values.
+enum Ordering : uint8 {
+  ASCENDING_THEN_NULLS,
+  DESCENDING_THEN_NULLS,
+  NULLS_THEN_ASCENDING,
+  NULLS_THEN_DESCENDING
+}
+
+/// An expression with an order
+table SortKey {
+  expression: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// Boundary is unbounded
+table Unbounded {}
+
+union ConcreteBoundImpl {
+  Expression,
+  Unbounded,
+}
+
+/// Boundary is preceding rows, determined by the contained expression
+table Preceding {
+  ipml: ConcreteBoundImpl (required);
+}
+
+/// Boundary is following rows, determined by the contained expression
+table Following {
+  impl: ConcreteBoundImpl (required);
+}
+
+/// Boundary is the current row
+table CurrentRow {}
+
+union BoundImpl {
+  Preceding,
+  Following,
+  CurrentRow,
+}
+
+/// Boundary of a window
+table Bound {
+  impl: BoundImpl (required);
+}
+
+/// The kind of window function to be executed.
+enum Frame : uint8 {
+  Rows,
+  Range,
+}
+
+/// An expression representing a window function call.
+table WindowCall {
+  /// The kind of window frame
+  kind: Frame;
+  /// The expression to operate over
+  expression: Expression (required);
+  /// Partition keys
+  partitions: [Expression] (required);
+  /// Sort keys
+  orderings: [SortKey] (required);
+  /// Lower window bound
+  lower_bound: Bound (required);
+  /// Upper window bound
+  upper_bound: Bound (required);
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalAggregateId : uint32 {
+  All,
+  Any,
+  Count,
+  CountTable,
+  Mean,
+  Min,
+  Max,
+  Product,
+  Sum,
+  Variance,
+  StandardDev,
+}
+
+
+table CanonicalAggregate {
+  id: CanonicalAggregateId;
+}
+
+table NonCanonicalAggregate {
+  name_space: string;
+  name: string (required);
+}
+
+union AggregateImpl {
+  CanonicalAggregate,
+  NonCanonicalAggregate,
+}
+
+table AggregateCall {
+  /// The kind of aggregate function being executed
+  kind: AggregateImpl (required);
+
+  /// Aggregate expression arguments
+  arguments: [Expression] (required);
+
+  /// Possible ordering.
+  orderings: [SortKey];
+
+  /// optional per-aggregate filtering
+  predicate: Expression;

Review comment:
       I'm not clear why this exists here? What specifically are we accomplishing that we can't accomplish outside this operation?

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);

Review comment:
       It's quite strange this is called default. In sql and programming it's called "else" :)
   

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {

Review comment:
       If you have a case, you probably should have both types. The if-then one and the switch case one. The switch case one has a bunch of nice optimizations and I'd prefer that the execution engine not have to identify those.

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// Output remapping of ordinals for a given operation
+  output_mapping: Emit (required);
+
+  /// Arguments for custom operations
+  options: InlineBuffer;
+}
+
+/// Filter operation
+table Filter {
+  base: RelBase (required);
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  predicate: Expression (required);
+}
+
+/// Projection
+table Project {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// Aggregate operation
+table Aggregate {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the aggregate relation's output.
+  aggregations: [Expression] (required);
+  /// Keys by which `aggregations` will be grouped.
+  keys: [Expression] (required);
+}
+
+enum CanonicalJoinKindId : uint8 {
+  Anti,
+  Cross,
+  FullOuter,
+  Inner,
+  LeftOuter,
+  LeftSemi,
+  RightOuter,
+  // TODO: Window
+  // TODO: AsOf
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string;
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind,
+  NonCanonicalJoinKind,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table Join {
+  base: RelBase (required);
+  /// The expression which will be evaluated against rows from each
+  /// input to determine whether they should be included in the
+  /// join relation's output.
+  on_expression: Expression (required);
+  /// The kind of join to use.
+  join_kind: JoinKind (required);
+}
+
+/// Order by relation
+table OrderBy {
+  base: RelBase (required);
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// Limit operation
+table Limit {
+  base: RelBase (required);
+  /// The maximum number of rows of output.
+  count: long;
+}
+
+/// Common table expresssion
+table Common {

Review comment:
       I wouldn't include this to start. While I can imagine some use cases, I feel like a higher level abstraction makes sense. For example Cache read and write nodes and then submitting multiple ordered plans (or similar)

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);

Review comment:
       I would avoid putting a generic arguments here. Many (most) operations only support a single input relation. Some support none. Having to write conditions all over making sure there is only the correct number of inputs weakens the spec. Let's only have an array when the value can 1..N. If 0,1,2 let's do those specifically.

##########
File path: format/experimental/computeir/ExampleExtensions.fbs
##########
@@ -0,0 +1,127 @@
+// 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 "../../Schema.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf.ext;
+
+/////////////////////////////////////////////////////////////
+// Example source relations:
+
+// RFC 3986, section 3
+table Uri {

Review comment:
       It's pretty confusing that this is part of this patch. I understand that is probably to be able to test out extensibility but I have some difficulty understanding why it isn't just a custom relation. Why is it partly decomposed/injected?

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+table Cast {
+  /// The expression to cast
+  expression: Expression (required);
+
+  /// The type to cast `argument` to.
+  type: org.apache.arrow.flatbuf.Field (required);

Review comment:
       I'm not convinced this sufficient. How does one express `cast('foo', VARCHAR(2))`?

##########
File path: format/experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,224 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT a, b FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [0, 1].
+table Remap {
+  mapping: [uint32] (required);
+}
+
+// Pass through indicates that no output remapping should occur.
+table PassThrough {}
+
+/// A union for the different colum remapping variants
+union Emit {
+  Remap,
+  PassThrough,
+}
+
+/// Fields common to every relational operator
+table RelBase {
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// Output remapping of ordinals for a given operation
+  output_mapping: Emit (required);
+
+  /// Arguments for custom operations
+  options: InlineBuffer;
+}
+
+/// Filter operation
+table Filter {
+  base: RelBase (required);
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  predicate: Expression (required);
+}
+
+/// Projection
+table Project {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// Aggregate operation
+table Aggregate {
+  base: RelBase (required);
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the aggregate relation's output.
+  aggregations: [Expression] (required);
+  /// Keys by which `aggregations` will be grouped.
+  keys: [Expression] (required);
+}
+
+enum CanonicalJoinKindId : uint8 {
+  Anti,
+  Cross,
+  FullOuter,
+  Inner,
+  LeftOuter,
+  LeftSemi,
+  RightOuter,
+  // TODO: Window
+  // TODO: AsOf
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string;
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind,
+  NonCanonicalJoinKind,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table Join {
+  base: RelBase (required);
+  /// The expression which will be evaluated against rows from each
+  /// input to determine whether they should be included in the
+  /// join relation's output.
+  on_expression: Expression (required);
+  /// The kind of join to use.
+  join_kind: JoinKind (required);
+}
+
+/// Order by relation
+table OrderBy {
+  base: RelBase (required);
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// Limit operation
+table Limit {
+  base: RelBase (required);
+  /// The maximum number of rows of output.
+  count: long;
+}
+
+/// Common table expresssion
+table Common {
+  base: RelBase (required);
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string (required);
+}
+
+/// The kind of set operation being performed.
+enum CanonicalSetOpKindId : uint8 {
+  Union,
+  Intersection,
+  Difference,
+}
+
+table CanonicalSetOpKind {
+  id: CanonicalSetOpKindId;
+}
+
+table NonCanonicalSetOpKind {
+  name_space: string;
+  name: string (required);
+}
+
+/// The kind of set operation
+union SetOpKind {
+  CanonicalSetOpKind,
+  NonCanonicalSetOpKind,
+}
+
+/// A set operation on two or more relations
+table SetOperation {
+  base: RelBase (required);
+  /// The kind of set operation
+  set_op: SetOpKind (required);
+}
+
+/// Literal relation
+table LiteralRelation {
+  base: RelBase (required);
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// A user-defined relation.
+table Custom {
+  base: RelBase (required);
+  /// NB: the schema is optional here, since a custom relation
+  /// may or may not need it.
+  schema: org.apache.arrow.flatbuf.Schema;

Review comment:
       I think we need more definition around Custom before introduce it. For example, how does custom declare physical property propagation or not?

##########
File path: format/experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,351 @@
+// 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 "../../Schema.fbs";
+include "Literal.fbs";
+include "InlineBuffer.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);
+}
+
+/// Struct field access
+table StructField {
+  /// The position of the field in the struct schema
+  position: uint32;
+}
+
+/// Zero-based array index
+table ArraySubscript {
+  position: uint32;
+}
+
+/// Zero-based range of elements in an array
+table ArraySlice {
+  /// The start of an array slice, inclusive
+  start_inclusive: uint32;
+  /// The end of an array slice, exclusive
+  end_exclusive: uint32;
+}
+
+/// Field name in a relation
+table FieldName {
+  position: uint32;
+}
+
+/// A union of possible dereference operations
+union Deref {
+  /// Access a value for a given map key
+  MapKey,
+  /// Access the value at a struct field
+  StructField,
+  /// Access the element at a given index in an array
+  ArraySubscript,
+  /// Access a range of elements in an array
+  ArraySlice,
+  /// Access a field of a relation
+  FieldName,
+}
+
+/// Access the data of a field
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  ref: Deref (required);
+  /// For Expressions which might reference fields in multiple Relations,
+  /// this index may be provided to indicate which Relation's fields
+  /// `path` points into. For example in the case of a join,
+  /// 0 refers to the left relation and 1 to the right relation.
+  relation_index: int;
+}
+
+/// A canonical (probably SQL equivalent) function
+//
+// TODO: variadics
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // date/time/timestamp operations
+  DateSub,
+  DateAdd,
+  DateDiff,
+  TimeAdd,
+  TimeSub,
+  TimeDiff,
+  TimestampAdd,
+  TimestampSub,
+  TimestampDiff,
+
+  // comparison
+  Equals,
+  NotEquals,
+  Greater,
+  GreaterEqual,
+  Less,
+  LessEqual,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string;
+  name: string (required);
+}
+
+union FunctionImpl {
+  CanonicalFunction,
+  NonCanonicalFunction,
+}
+
+/// A function call expression
+table Call {
+  /// The kind of function call this is.
+  kind: FunctionImpl (required);
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  when: Expression (required);
+  then: Expression (required);
+}
+
+/// Case statement-style expression.
+table Case {
+  cases: [CaseFragment] (required);
+  /// The default value if no cases match. This is typically NULL in SQL
+  //implementations.
+  ///
+  /// Defaulting to NULL is a frontend choice, so producers must specify NULL
+  /// if that's their desired behavior.
+  default: Expression (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  metadata: InlineBuffer;

Review comment:
       I won't put this everywhere but I think we need to remove all these embedded buffers.

##########
File path: format/experimental/computeir/InlineBuffer.fbs
##########
@@ -0,0 +1,58 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       I don't see a good reason why any of this file should exist. I suggest we instead pattern after something like this:
   
   ```
   union Relation {
     Aggregate,
     Join,
     ...
    CustomRelation
   }
   
   // similar to Protobuf Any concepts, which actually have language bindings.
   table Any { 
     type: string // a flatbuf namespace table name. For example "com.mycompany.ir.CustomFancyJoin"
     data: bytes; // the serialized definition of the type above.
   }
   
   table CustomRelation {
     base: BaseRel;
     details: Any;
   }
   ```
   
   That way things are:
   - Self-describing
   - Can be machine validated
   
   I'd also avoid sprinkling in non-canonical all over the place unless there is a real need. For example, why would I have a "non-canonical set op" as opposed to just having a new CustomRelation?




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