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/13 18:50:40 UTC

[GitHub] [arrow] bkietz opened a new pull request #10934: Open [RFC] Arrow Compute Serialized Intermediate Representation draft for discussion

bkietz opened a new pull request #10934:
URL: https://github.com/apache/arrow/pull/10934


   See also #10856


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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712422002



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {
+  SECOND,
+  MILLISECOND,
+  MICROSECOND,
+  NANOSECOND,
+}
+
+table TimeLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+table TimestampLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit;
+
+  // timezone value, same definition as used in Schema.fbs.
+  timezone: string;
+}
+
+table IntervalLiteralMonths {
+  months: int32;
+}
+
+table IntervalLiteralDaysMilliseconds {
+  days: int32;
+  milliseconds: int32;
+}

Review comment:
       Unresolving this to get your thoughts. It's likely that substrait will not have support for nanoseconds here. If we add it in arrow, and then write a bunch of code that accepts nanos intervals I don't think we can move to substrait without a ton of hassle and breakage. I think it would make sense to add this only if we absolutely need it. Maybe we will need it immediately and I am just unaware.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689807331



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape

Review comment:
       I don't think so. I'll remove it




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697520365



##########
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:
       Can a user express this kind of collation? E.g., `SELECT k, mean(v) FROM t GROUP BY k CLUSTER BY k` or something similar?




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689823790



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;

Review comment:
       Yes, they are [defined as](https://github.com/bkietz/arrow/blob/a7ca419539ac11f6fada9dcc94ad5859d06760a6/format/Schema.fbs#L398-L410)
   
   ```c++
   /// A Buffer represents a single contiguous memory segment
   struct Buffer {
     /// The relative offset into the shared memory page where the bytes for this
     /// buffer starts
     offset: long;
   
     /// The absolute length (in bytes) of the memory buffer. The memory is found
     /// from offset (inclusive) to offset + length (non-inclusive). When building
     /// messages using the encapsulated IPC message, padding bytes may be written
     /// after a buffer, but such padding bytes do not need to be accounted for in
     /// the size here.
     length: long;
   }
   ```
   
   IE they reference memory which is out-of-band WRT flatbuffers. This gives producers of Buffers control over padding and alignment, which `[ubyte]` does not




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697512149



##########
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:
       Sure, removed  it.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697508915



##########
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:
       Primitives default to 0 IIRC, but I've made it explicit.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697594246



##########
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:
       Then the modelling of a non-expression argument isn't necessary.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691282786



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);

Review comment:
       SGTM




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689620571



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;

Review comment:
       For example, providing a [`float_ord` function](https://gist.github.com/bkietz/8e2ef182883b886e532ffde8e537f7a3) which maps floats onto a type with less nuanced ordering could accommodate all of these cases




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697553655



##########
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:
       This to model extracting parts of date/time/timestamp operations:  eg https://cloud.google.com/bigquery/docs/reference/standard-sql/date_functions#extract




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697522700



##########
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?
   
   Is there any impact on consumers here? For example, if a producer must specify this, then a consumer must follow it, whereas maybe a query that leaves null ordering unspecified is much more performant because it doesn't have to sort.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697510589



##########
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:
       You cannot use `union`s in flatbuffers directly, they have to be wrapped in a `table`.
   
   From the flatbuffers docs:
   
   > Note that because a union field is really two fields, it must always be part of a table, it cannot be the root of a FlatBuffer by itself.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697530861



##########
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:
       `predicate` isn't necessary, since it can be modeled as a `case` expression. I'll remove it




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712394596



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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 = 0;
+}
+
+/// A function call expression
+table Call {
+  /// The function to call
+  name: string (required);
+
+  /// The arguments passed to `name`.
+  arguments: [Expression] (required);
+
+  /// Possible ordering of input. These are useful
+  /// in aggregates where ordering in meaningful such as
+  /// string concatenation
+  orderings: [SortKey];
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  match: Expression (required);
+  result: Expression (required);
+}
+
+/// Conditional case statement expression
+table ConditionalCase {

Review comment:
       Mostly to provide consumers with more structure if they want it, though the choice of this for case and not for everything is somewhat arbitrary. Can we leave it as is and if it turns out not be necessary we'll move it to a `Call` expression?




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712394935



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);

Review comment:
       Hm, I thought they were. I'll do another pass over the `required`s.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697550529



##########
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:
       Removed.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697597224



##########
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:
       Right, I should've said explicitly that the above means it would go away as a first class operation.




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



[GitHub] [arrow] github-actions[bot] commented on pull request #10934: Open [RFC] Arrow Compute Serialized Intermediate Representation draft for discussion

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-898655955


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/master/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on JIRA? https://issues.apache.org/jira/browse/ARROW
   
   Opening JIRAs ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697567120



##########
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 would remove. The Extract date syntax is syntactic sugar that maps to a particular set of functions/arguments. 
   
   FWIW, cast is much the same. The only reason I thought it made sense to keep cast as an independent concept as opposed to just a function is there are some situations where you want to walk across casts to do certain optimizations and having specialized types cleans that up. However, if we think more at the serialization level, maybe it makes to remove cast as well.




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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690580989



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);

Review comment:
       what this indirection?




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



[GitHub] [arrow] github-actions[bot] commented on pull request #10934: [RFC] Arrow Compute Serialized Intermediate Representation draft for discussion

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-898710688


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/master/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on JIRA? https://issues.apache.org/jira/browse/ARROW
   
   Opening JIRAs ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712416330



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+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 {
+  /// Output remapping of ordinal columns for a given operation
+  output_mapping: Emit (required);
+}
+
+/// Filter operation
+table Filter {

Review comment:
       Scalar and aggregate functions, yes.




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712429582



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {
+  SECOND,
+  MILLISECOND,
+  MICROSECOND,
+  NANOSECOND,
+}
+
+table TimeLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+table TimestampLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit;
+
+  // timezone value, same definition as used in Schema.fbs.
+  timezone: string;
+}
+
+table IntervalLiteralMonths {
+  months: int32;
+}
+
+table IntervalLiteralDaysMilliseconds {
+  days: int32;
+  milliseconds: int32;
+}

Review comment:
       Fair enough. We can add it later.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712389668



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+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 {
+  /// Output remapping of ordinal columns for a given operation
+  output_mapping: Emit (required);
+}
+
+/// Filter operation
+table Filter {

Review comment:
       `Filter` is a relation, not a column expression, and its one of the fundamental relations, so I do think it's necessary.




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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698867003



##########
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:
       Please take this with the usual caveat that I am no expert in this field.
   
   > A producer and a consumer that are not self-contained. This is the broadest use case and I think will be the primary way in which the IR is used. In this scenario, the producer inserts a type into this field as the output of its type system. The consumer is then required to adhere to what the producer asked for, or return an error.
   
   This sounds like the user needs to create overloads based solely on the return type.  I feel like that will be rare enough that it can be handled via the name or the call arguments/options.  For example, `add_unchecked(int32, int32) => int32` and `add_or_widen(int32, int32) => int64`.  It would be odd for me to pick which behavior I get by specifying the return type.
   
   > A producer and consumer that are self-contained. This is for a system such as a relational database (e.g., DuckDB), where the IR is being used by a system that controls both the producer and the consumer. The purpose of the type field here is to have a place for the output of a type derivation step to be stored after its derivation for later consumption.
   
   This feels like a slippery slope.  Wouldn't it be better for DuckDB in this case to extend the fbs with their own fields?
   
   




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r699558058



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

Review comment:
       Added.




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



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

Posted by GitBox <gi...@apache.org>.
kkraus14 commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697787664



##########
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 may also need an enum for the ordering of `NaN` with respect to `null`. I.E. is it `NaN` then `null` then ascending values, or `null` then `NaN` then ascending values?




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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698867003



##########
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:
       Please take this with the usual caveat that I am no expert in this field.
   
   > A producer and a consumer that are not self-contained. This is the broadest use case and I think will be the primary way in which the IR is used. In this scenario, the producer inserts a type into this field as the output of its type system. The consumer is then required to adhere to what the producer asked for, or return an error.
   
   This sounds like the user needs to create overloads based solely on the return type.  I feel like that will be rare enough that it can be handled via the name or the call arguments/options.  For example, `add_unchecked(int32, int32) => int32` and `add_or_widen(int32, int32) => int64`.  It would be odd for me to pick which behavior I get by specifying the return type.
   
   > A producer and consumer that are self-contained. This is for a system such as a relational database (e.g., DuckDB), where the IR is being used by a system that controls both the producer and the consumer. The purpose of the type field here is to have a place for the output of a type derivation step to be stored after its derivation for later consumption.
   
   This feels like a slippery slope.  Wouldn't it be better for DuckDB in this case to extend the fbs with their own fields?
   
   




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688740401



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+table Literal {
+  /// Number of slots. If this is absent, the literal should be
+  /// considered scalar.
+  length: long;
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+enum JoinKind : int {
+  INNER,
+  LEFT,
+  RIGHT,
+  FULL,
+  SEMI,
+  ANTI,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: JoinKind;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.name = "literal"
+table LiteralOptions {
+  /// Batches of rows in this literal.
+  batches: [RecordBatch] (required);
+  /// A dictionary batch for this literal.
+  dictionary_batch: DictionaryBatch;
+}
+
+/// The contents of Relation.options will be InteractiveOutputOptions
+/// if Relation.name = "interactive_output"
+table InteractiveOutputOptions {
+  /// In an interactive context there is an implicit pipe open between
+  /// the producer and the consumer along which output may be sent.
+  /// `id` is an opaque identifier for such outputs.

Review comment:
       ```suggestion
     /// `id` is an opaque identifier for such outputs; for example it could be used to
     /// identify which Jupyter output cell should receive the output.
   ```




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712404835



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {
+  SECOND,
+  MILLISECOND,
+  MICROSECOND,
+  NANOSECOND,
+}
+
+table TimeLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+table TimestampLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit;
+
+  // 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 DurationLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+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.

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712387417



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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

Review comment:
       Yup, it's old.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689836542



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;
+}
+
+/// The contents of Relation.options will be UnionOptions
+/// if Relation.relation_name = "union"
+table UnionOptions {
+  /// For simplicity, all rows from any input to a "union" relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.relation_name = "literal"
+table LiteralOptions {
+  /// Batches of rows in this literal.
+  batches: [RecordBatch] (required);
+}
+
+/// A specification of a query.
+table Plan {
+  /// One or more output relations.
+  sinks: [Relation] (required);
+
+  /// Dictionary batches which may be referenced by Literals.
+  dictionary_batch: [DictionaryBatch];

Review comment:
       Some `Literal`s (expressions here, not literal relations) may be dictionary encoded, in which case they need to refer into a dictionary batch (this organization is inherited from the arrow IPC format). I could instead move this back to a member of LiteralOptions, but I'd need to add it as a member of Literal (expression) as well




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697593674



##########
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:
       That's true, but then there's a need to model arguments that aren't expressions since `CAST` takes a type as an "argument". One way to do that in the current scheme is to use the `type` field of `Expression`.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691301659



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.operation = CanonicalOperation::Filter
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.operation = CanonicalOperation::Project
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.operation = CanonicalOperation::Aggregate
+table AggregateOptions {
+  /// 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 : uint32 {
+  Inner,
+  LeftOuter,
+  RightOuter,
+  FullOuter,
+  Cross,
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string (required);
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind, NonCanonicalJoinKind
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table JoinOptions {
+  /// 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);
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.operation = CanonicalOperation::OrderBy
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.operation = CanonicalOperation::Limit
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.operation = CanonicalOperation::Common
+table CommonOptions {
+  /// 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 contents of Relation.options will be UnionOptions
+/// if Relation.operation = CanonicalOperation::Union
+table UnionOptions {
+  /// For simplicity, all rows from any input to a union relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.operation = CanonicalOperation::Literal
+table LiteralOptions {
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// A specification of a query.
+table Plan {
+  /// One or more output relations.
+  sinks: [Relation] (required);

Review comment:
       This is a difference in execution model for this proposal (for which `rpc_service Interactive` is provided semi-didactically to clarify).
   
   If the root_type is a TableExpr which evaluates to batches, that implies that there is a channel open between the consumer and the producer along which those batches can be returned. This is frequently the case but in general I think we'll want to be able to express execution plans which don't rely on an interactive channel. For example: Plans generated as fragments of larger plans in distributed execution or plans which represent an ETL job.
   
   Therefore I think it's preferable that a Plan explicitly include the destination for all batches, even if that will quite commonly be `operation=InteractiveOutput` (just pipe them back to the user).
   
   Only a single instance of InteractiveOutput is permitted in a Plan (so no interactive producer will need to deinterleave batches piped back along the interactive channel, which I *think* was your concern here). However any number of other sinks are permitted. For example, a Plan may specify that one set of batches be streamed to `tcp://somehost.com:890` for consumption by a service on that host and an unfiltered superset of those batches cached locally into `file://tmp/cache/my_query` for debugging.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688834402



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}

Review comment:
       Again, I'd defer this configuration to a function call rather than introduce more configuration




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712417670



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+table Remap {
+  mapping: [uint32] (required);

Review comment:
       Sorry, this branch is a bit long-lived :sweat_smile: . Renaming to `FieldIndex`.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688754386



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];

Review comment:
       ```suggestion
     bytes: [ubyte] (required);
   ```

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+table Literal {
+  /// Number of slots. If this is absent, the literal should be
+  /// considered scalar.
+  length: long;
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];

Review comment:
       ```suggestion
     path: [string] (required);
   ```




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712390855



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+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 {
+  /// Output remapping of ordinal columns for a given operation
+  output_mapping: Emit (required);
+}
+
+/// Filter operation
+table Filter {

Review comment:
       Ah, I see, a `Call` can only refer to what Arrow C++ names scalar functions, right?




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712353459



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  values: [Literal];
+}
+
+table StructLiteral {
+  values: [KeyValue];
+}
+
+table KeyValue {
+  key: string;
+  value: Literal;
+}
+
+table MapLiteral {
+  values: [KeyValue];

Review comment:
       `KeyValue` forces the key to be a string but a map can accept any type as key type, is it deliberate to restrict MapLiteral to string keys?

##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {

Review comment:
       `Schema.fbs` already has `DateUnit` and `TimeUnit`, is the redefinition deliberate?

##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;

Review comment:
       Is it deliberate to use `uint8` for scale and precision rather than `int` as in `Decimal`?
   
   At least the scale should probably be signed -- negative scales are allowed in Arrow, even though they may not be very compatible with other systems.

##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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

Review comment:
       Or is it me, or is this comment outdated?

##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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,

Review comment:
       What is `path` here?

##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);

Review comment:
       Note: these files should be consistent in their use of `(required)`.

##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+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 {
+  /// Output remapping of ordinal columns for a given operation
+  output_mapping: Emit (required);
+}
+
+/// Filter operation
+table Filter {
+  /// Common options
+  base: RelBase (required);
+  /// Child relation
+  rel: Relation (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 {

Review comment:
       Same question obviously :-)

##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];

Review comment:
       Is this little-endian encoded?

##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {
+  SECOND,
+  MILLISECOND,
+  MICROSECOND,
+  NANOSECOND,
+}
+
+table TimeLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+table TimestampLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit;
+
+  // 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 DurationLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+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.

Review comment:
       No fixed-size binary literal either?

##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {
+  SECOND,
+  MILLISECOND,
+  MICROSECOND,
+  NANOSECOND,
+}
+
+table TimeLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+table TimestampLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit;
+
+  // timezone value, same definition as used in Schema.fbs.
+  timezone: string;
+}
+
+table IntervalLiteralMonths {
+  months: int32;
+}
+
+table IntervalLiteralDaysMilliseconds {
+  days: int32;
+  milliseconds: int32;
+}

Review comment:
       You probably want to add a `IntervalLiteralDaysMonthsNanos` as well.

##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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 = 0;
+}
+
+/// A function call expression
+table Call {
+  /// The function to call
+  name: string (required);
+
+  /// The arguments passed to `name`.
+  arguments: [Expression] (required);
+
+  /// Possible ordering of input. These are useful
+  /// in aggregates where ordering in meaningful such as
+  /// string concatenation
+  orderings: [SortKey];
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  match: Expression (required);
+  result: Expression (required);
+}
+
+/// Conditional case statement expression
+table ConditionalCase {

Review comment:
       Is there a reason not to express this as a `Call` with a well-defined function name?
   

##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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 = 0;
+}
+
+/// A function call expression
+table Call {
+  /// The function to call
+  name: string (required);

Review comment:
       Is it intended to normalize function names for the most common functions at some point? (e.g. `add` etc.)

##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+table Remap {
+  mapping: [uint32] (required);

Review comment:
       Is this desired instead of a bunch of `FieldName` expressions?

##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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 = 0;
+}
+
+/// A function call expression
+table Call {
+  /// The function to call
+  name: string (required);
+
+  /// The arguments passed to `name`.
+  arguments: [Expression] (required);
+
+  /// Possible ordering of input. These are useful
+  /// in aggregates where ordering in meaningful such as
+  /// string concatenation
+  orderings: [SortKey];
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  match: Expression (required);
+  result: Expression (required);
+}
+
+/// Conditional case statement expression
+table ConditionalCase {
+  /// List of conditions to evaluate
+  conditions: [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.
+  else: Expression (required);
+}
+
+/// Switch-style case expression
+table SimpleCase {

Review comment:
       Same question.

##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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 = 0;
+}
+
+/// A function call expression
+table Call {
+  /// The function to call
+  name: string (required);
+
+  /// The arguments passed to `name`.
+  arguments: [Expression] (required);
+
+  /// Possible ordering of input. These are useful
+  /// in aggregates where ordering in meaningful such as
+  /// string concatenation
+  orderings: [SortKey];
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  match: Expression (required);
+  result: Expression (required);
+}
+
+/// Conditional case statement expression
+table ConditionalCase {
+  /// List of conditions to evaluate
+  conditions: [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.
+  else: Expression (required);
+}
+
+/// Switch-style case expression
+table SimpleCase {
+  /// The expression whose value will be matched
+  expression: Expression (required);
+  /// Matches for `expression`
+  matches: [CaseFragment] (required);
+  /// The default value if no cases match
+  else: Expression (required);
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values
+enum Collation : uint8 {

Review comment:
       "Collation" sounds more like the kind of comparison function you're going to apply (such as unicode collation)?

##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+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 {
+  /// Output remapping of ordinal columns for a given operation
+  output_mapping: Emit (required);
+}
+
+/// Filter operation
+table Filter {

Review comment:
       Can this be simply be expressed as a `Call`? Do we benefit from having a dedicated `Filter` node?




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



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

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712398525



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+table Remap {
+  mapping: [uint32] (required);

Review comment:
       Well, `FieldName` doesn't take a name, it takes a `uint32`. Perhaps you want to change its name :-)




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698458633



##########
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:
       Are we actually sure this is possible to do for all of the types we want to support?
   
   What the key is for, say, the `==` operator for a complex type like list or struct?
   
   I don't think a wildcard type is well-defined here without more clarification. For example, `List<T>` can only be compared with `List<U>` if `T == U`, but if `T != U` the operation is undefined.
   
   Unnest is another example.
   
   Without a type system that handles generics, you can't write down the type of all possible instantiations of any type that has a type parameter, such as list, map, and struct.
   
   What is the issue with having a list of functions in some structured format, that indicates the canonical name of the function and its arity?
   
   If a producer sends over a call to the add function with input types `int32, int32` and output type `int32`, then the consumer would look that up, and if it's able to execute that IR, then it does and if it's not able to do so it returns an error.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697041056



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction

Review comment:
       Per my comments elsewhere, if we want to support serialized functions, we should make them structured, not just a bucket of bytes. For example:
   {
     type: "python_pickle", 
     argument_types: [int], 
     output_type:"int", 
     dynamic:"false", 
     maintains: [sort, cluster, distribution],  
     python_prerequisites: [pyarrow, pytorch],
     bytes:<bytes>"
   }
   
   The structure should be such that: 
   - tools don't have to know how to decode the bytes to do plan transformations
   - transformations can move operations without having to worry about invalidating correctness (for example move where distribution of data is happening in a plan)
   - the plan can be validated without doing a byte decoding




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698475304



##########
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 think the purpose of this field is potentially being misunderstood.
   
   There are two very broad use cases that this is designed for:
   
   1. A producer and a consumer that are not self-contained. This is the broadest use case and I think will be the primary way in which the IR is used. In this scenario, the producer inserts a type into this field as the output of its type system. The consumer is then required to adhere to what the producer asked for, or return an error.
   2. A producer and consumer that are self-contained. This is for a system such as a relational database (e.g., DuckDB), where the IR is being used by a system that controls both the producer and the consumer. The purpose of the type field here is to have a place for the output of a type derivation step to be stored after its derivation for later consumption.
   
   Additionally, the field is optional to allow for the type derivation phase to happen at any point in time.
   
   What I think would be a mistake is to not have this field here and assume the producer and consumer will necessarily derive the same output type for every expression. We've already said that defining output type derivation rules are the responsibility of producers. Are we changing that decision?
   
   Regarding performance, I don' t think that's something we should focus on until we're happy with the design that supports the use cases we want to support, and until we have some clarity on what is actually expensive in real-world use cases.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689827714



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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:
       Cool, a producer can always orthogonalize the API if it so desires.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691402208



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       I've tried an approach inspired by `arrow2`'s decision to include the primitive type being stored in buffers (and thus also alignment information) all the way down to `struct Bytes`
   
   https://github.com/jorgecarleitao/arrow2/blob/main/src/buffer/bytes.rs#L39
   
   I *think* this guarantees alignment without requiring padding fields or reinterpret_casts




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689823790



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;

Review comment:
       Yes, they are defined as
   
   ```c++
   /// A Buffer represents a single contiguous memory segment
   struct Buffer {
     /// The relative offset into the shared memory page where the bytes for this
     /// buffer starts
     offset: long;
   
     /// The absolute length (in bytes) of the memory buffer. The memory is found
     /// from offset (inclusive) to offset + length (non-inclusive). When building
     /// messages using the encapsulated IPC message, padding bytes may be written
     /// after a buffer, but such padding bytes do not need to be accounted for in
     /// the size here.
     length: long;
   }
   ```
   
   IE they reference memory which is out-of-band WRT flatbuffers. This gives producers of Buffers control over padding and alignment, which `[ubyte]` does not




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



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

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698818377



##########
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:
       Why not follow the IEEE754 recommendation?
   
   * Iceberg uses it: https://github.com/apache/iceberg/blob/master/site/docs/spec.md#sorting
   * Java uses it
   * Rust lang follows it in practice (see e.g. https://github.com/rust-lang/rust/issues/5585 and [ord-float](https://crates.io/crates/float-ord))




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712388045



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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 = 0;
+}
+
+/// A function call expression
+table Call {
+  /// The function to call
+  name: string (required);

Review comment:
       I think so, but we'd like to punt on the function registry for now. It's a very large can of worms that I think will be better solved in the subtrait project.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712407596



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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 = 0;
+}
+
+/// A function call expression
+table Call {
+  /// The function to call
+  name: string (required);
+
+  /// The arguments passed to `name`.
+  arguments: [Expression] (required);
+
+  /// Possible ordering of input. These are useful
+  /// in aggregates where ordering in meaningful such as
+  /// string concatenation
+  orderings: [SortKey];
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  match: Expression (required);
+  result: Expression (required);
+}
+
+/// Conditional case statement expression
+table ConditionalCase {
+  /// List of conditions to evaluate
+  conditions: [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.
+  else: Expression (required);
+}
+
+/// Switch-style case expression
+table SimpleCase {
+  /// The expression whose value will be matched
+  expression: Expression (required);
+  /// Matches for `expression`
+  matches: [CaseFragment] (required);
+  /// The default value if no cases match
+  else: Expression (required);
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values
+enum Collation : uint8 {

Review comment:
       Changed to `Ordering` and removed `CLUSTERED`.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697594449



##########
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've removed `Custom`.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697944009



##########
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'd suggest going much simpler: we simply have castint, castvarchar, etc as functions. There is no need for a type argument. They are really just completely different functions that have nothing to do with each other. We should try to avoid coupling sql syntactic sugar with what is expressed in the plan.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697944069



##########
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'd suggest we start with just having a read a path of parquet files to start. It seems like a simple set of properties and a realistic initial usecase.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690593891



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;
+}
+
+/// The contents of Relation.options will be UnionOptions
+/// if Relation.relation_name = "union"
+table UnionOptions {
+  /// For simplicity, all rows from any input to a "union" relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.

Review comment:
       Thanks for the response!




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712405475



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697531534



##########
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:
       > Can a user express this kind of collation? E.g., SELECT k, mean(v) FROM t GROUP BY k CLUSTER BY k or something similar?
   
   Not traditionally. This is more about the physical side where certain operations expect clustering, such as Streaming Aggregate and writing where you are trying to only have one partition open at a time per stream.
   
   > Is there any impact on consumers here? For example, if a producer must specify this, then a consumer must follow it, whereas maybe a query that leaves null ordering unspecified is much more performant because it doesn't have to sort.
   
   Yeah. On one hand you could say producer is agnostic and engine should do whatever is most efficient. On the other hand, you could say that a producer should produce even if the user expresses no preference. Most platforms will default to one or the other if the user doesn't specify, I believe.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697944747



##########
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:
       >> But that is a massive scope increase for this project to 
   
   I guess I don't really see this as a massive increase in scope.  Maybe it just about being more formal sooner? Without being formal about this I fear that this will become a singly used representation and the implementation will define the specification, rather than the other way around.
   
   >> ...enumerate all the function signatures of every function overload contemplated...
   Part of it to me is also that this will be a growing list. It doesn't have to start with all possible functions, only the functions we want to initially specify. I would expect adding new functions would be relatively straightforward.
   
   >> I don't follow why operations with multiple overloads need to be dealt with at all in the IR. Wouldn't a function have a singular definition (or be singularly derivable) for a given IR?
   From an engine implementation point of view, as an example, I feel like decimal division has much more in common with decimal multiplication than it does with integer division. It also have a very different type output resolution system. Overloading a single concept of division to state different output type derivation systems seems quite a bit more complex than simply saying that there is no "overloading". To me, we simply need to consider each function's key to be the name + the input argument types. Sure, two functions may have the same "name" but that doesn't mean they have the same key (or have anything to do with each other).
   
   




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690584137



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);

Review comment:
       The generators for some languages don't support vector-of-unions. I'll add a comment explaining that

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+table Call {
+  /// The namespaced name of the function whose invocation this Call represents.
+  /// For example: "arrow::add" or "gandiva::jit_3432".
+  ///
+  /// Names with no namespace are reserved for canonicalization.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  /// For example: "arrow::hash_join" or "gandiva::filter_and_project".
+  ///
+  /// Names with no namespace are reserved for canonical, "pure" relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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);
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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 namespaced name of the join to use. Non-namespaced names are
+  /// reserved for canonicalization. Current names include:
+  ///   "inner"
+  ///   "left"
+  ///   "right"
+  ///   "outer"
+  ///   "cross"
+  join_name: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream

Review comment:
       I'll expand the comment

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       I think this can be resolved by taking a leaf out of parquet2's book and defining InlineBuffer as a union of vectors of each primitive type, I'll try that.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);

Review comment:
       SGTM

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction

Review comment:
       I'm not sure how that's distinct from providing a NonCanonicalFunction with the serialized function in the options blob

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.operation = CanonicalOperation::Filter
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.operation = CanonicalOperation::Project
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.operation = CanonicalOperation::Aggregate
+table AggregateOptions {
+  /// 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 : uint32 {
+  Inner,
+  LeftOuter,
+  RightOuter,
+  FullOuter,
+  Cross,
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string (required);
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind, NonCanonicalJoinKind
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table JoinOptions {
+  /// 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);
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.operation = CanonicalOperation::OrderBy
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.operation = CanonicalOperation::Limit
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.operation = CanonicalOperation::Common
+table CommonOptions {
+  /// 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 contents of Relation.options will be UnionOptions
+/// if Relation.operation = CanonicalOperation::Union
+table UnionOptions {
+  /// For simplicity, all rows from any input to a union relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.operation = CanonicalOperation::Literal
+table LiteralOptions {
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// A specification of a query.
+table Plan {
+  /// One or more output relations.
+  sinks: [Relation] (required);

Review comment:
       This is a difference in execution model for this proposal (for which `rpc_service Interactive` is provided semi-didactically to clarify).
   
   If the root_type is a TableExpr which evaluates to batches, that implies that there is a channel open between the consumer and the producer along which those batches can be returned. This is frequently the case but in general I think we'll want to be able to express execution plans which don't rely on an interactive channel. For example: Plans generated as fragments of larger plans in distributed execution or plans which represent an ETL job.
   
   Therefore I think it's preferable that a Plan explicitly include the destination for all batches, even if that will quite commonly be `operation=InteractiveOutput` (just pipe them back to the user).
   
   Only a single instance of InteractiveOutput is permitted in a Plan (so no interactive producer will need to deinterleave batches piped back along the interactive channel, which I *think* was your concern here). However any number of other sinks are permitted. For example, a Plan may specify that one set of batches be streamed to `tcp://somehost.com:890` for consumption by a service on that host and an unfiltered superset of those batches cached locally into `file://tmp/cache/my_query` for debugging.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;

Review comment:
       -1. This adds yet more special casing for the canonical operations and makes it harder to write generic pattern matching utilities while also giving us another discriminant to query and validate.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);

Review comment:
       I agree that every Expression needs type, but I disagree that name is meaningful for Expressions. To my mind a name is ascribed to the referent by the referring entity; to speak in graph theory it's an edge property rather than a node property. To give a practical context: in a projection like `SELECT $complicated_expr as A, $complicated_expr as B` it should not be an error to use a single memoized instance of `$complicated_expr`- but including name as an expression property would make these two incompatible.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       I've tried an approach inspired by `arrow2`'s decision to include the primitive type being stored in buffers (and thus also alignment information) all the way down to `struct Bytes`
   
   https://github.com/jorgecarleitao/arrow2/blob/main/src/buffer/bytes.rs#L39
   
   I *think* this guarantees alignment without requiring padding fields or reinterpret_casts




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



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

Posted by GitBox <gi...@apache.org>.
kkraus14 commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688802634



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}

Review comment:
       Maybe we should clarify whether this is irrespective of `Ordering`?
   
   If it's not, then would maybe suggest changing to `GREATEST` and `LEAST`.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.

Review comment:
       ```suggestion
   /// A relation is a set of rows with consistent schema.
   ```

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}

Review comment:
       Do we need the ability to control `null` and/or `NaN` behavior here? Are they always grouped together?

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}

Review comment:
       Do we need the ability to control whether `null` and/or `NaN` values are considered equal here?

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;

Review comment:
       Do we need to be able to control `NaN` ordering as well for floating point columns? And `NaN` versus `null` ordering?




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697524963



##########
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:
       The `(required)` here just means that there must be something here. An empty array is a valid something.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697512982



##########
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:
       Roger, changing to `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 {
+  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:
       Roger, changing to `else`




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689828608



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;

Review comment:
       As with custom function and relation names, it's up to the consumer to decide if the names are valid. It would probably be worthwhile to include a listing of reserved names and their canonical semantics, ala
   
   ```
   "inner"
   "left"
   "right"
   "outer"
   "cross"
   ...
   ```




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697524485



##########
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:
       The idea here is that the wrapper is the primary conceptual object and the impls are the concrete instantiations of the type. I don't really like the idea of renaming everything with a `Wrapper` suffix, mostly because I think it's confusing to read, but if it generates more cognitive dissonance to use `Impl`, then happy to change it.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697594631



##########
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:
       Fixed.




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



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

Posted by GitBox <gi...@apache.org>.
kkraus14 commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689142166



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;

Review comment:
       NaN coercian doesn't solve the problem. I believe there's use cases out there that need to control specific ordering with regards to `NaN` vs `null` vs other values. I.E. you could imagine all of the following being a desired output for an ascending sort:
   - `[NaN, null, 1.0, 2.0, 5.0]`
   - `[null, NaN, 1.0, 2.0, 5.0]`
   - `[NaN, 1.0, 2.0, 5.0, null]`
   - `[null, 1.0, 2.0, 5.0, NaN]`
   - `[NaN, 1.0, 2.0, 5.0, null]`
   - `[1.0, 2.0, 5.0, NaN, null]`
   - `[1.0, 2.0, 5.0, null, NaN]`
   
   I think to accomplish this you need something like `null_ordering`, `nan_ordering`, and `nan_vs_null` parameters.
   
   If this could be baked into the `value` Expression in some form then that's fine.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r699558188



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

Review comment:
       Added `DurationLiteral`.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697506852



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction

Review comment:
       I think that makes sense. We'll have to a lot more legwork to make user-defined structures a thing.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697552545



##########
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:
       One way might be to add the `2` into the `custom_metadata` field of the `Field` instance.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691362576



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;

Review comment:
       -1. This adds yet more special casing for the canonical operations and makes it harder to write generic pattern matching utilities while also giving us another discriminant to query and validate.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697675703



##########
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:
       In that case, I think there's probably an argument to split out these options into 3 different enums, one for non-null ordering, null ordering, and nan ordering.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697583544



##########
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:
       I can remove these examples. They were based on an older design, which is different from the current iteration.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712390539



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+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 {
+  /// Output remapping of ordinal columns for a given operation
+  output_mapping: Emit (required);
+}
+
+/// Filter operation
+table Filter {
+  /// Common options
+  base: RelBase (required);
+  /// Child relation
+  rel: Relation (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 {

Review comment:
       Same answer :), though I see your point. I think to make this a function we'd have to treat tables as functions that produce `struct`s, which would be a pretty big change here I think.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697510960



##########
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:
       I suppose that we could just ship the union around, and avoid the wrapper.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-906810077


   Additional comments: given this interest in targeting physical engines, it is weird that we are missing common physical operations like HashJoin, MergeJoin, HashAggregate, StreamingAggregate, TopN (order + limit operator), as well as a bunch related to RedistributeSend, OrderedRedistribute, UnorderedReception, OrderedReception, etc. 
   
   I also think we need to go to each operation and declare the following:
   - Properties each operation maintains. (e.g. if you have Read > Sort > Filter > Write, is the write expected to be sorted?)
   - The order of the output fields/columns. For example, Join (A, B), Project(a+b, c+d), etc.
   


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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697592432



##########
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:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690593891



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;
+}
+
+/// The contents of Relation.options will be UnionOptions
+/// if Relation.relation_name = "union"
+table UnionOptions {
+  /// For simplicity, all rows from any input to a "union" relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.

Review comment:
       Thanks for the response!




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698458633



##########
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:
       Are we actually sure this is possible to do for all of the types we want to support?
   
   What the key is for, say, the `==` operator for a complex type like list or struct?
   
   I don't think a wildcard type is well-defined here without more clarification. For example, `List<T>` can only be compared with `List<U>` if `T == U`, but if `T != U` the operation is undefined.
   
   Unnest is another example.
   
   With a type system that handles generics, you can't write down the type of all possible instantiations of any type that has a type parameter, such as list, map, and struct.
   
   What is the issue with having a list of functions in some structured format, that indicates the canonical name of the function and its arity?
   
   If a producer sends over a call to the add function with input types `int32, int32` and output type `int32`, then the consumer would look that up, and if it's able to execute that IR, then it does and if it's not able to do so it returns an error.

##########
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:
       Are we actually sure this is possible to do for all of the types we want to support?
   
   What the key is for, say, the `==` operator for a complex type like list or struct?
   
   I don't think a wildcard type is well-defined here without more clarification. For example, `List<T>` can only be compared with `List<U>` if `T == U`, but if `T != U` the operation is undefined.
   
   Unnest is another example.
   
   Without a type system that handles generics, you can't write down the type of all possible instantiations of any type that has a type parameter, such as list, map, and struct.
   
   What is the issue with having a list of functions in some structured format, that indicates the canonical name of the function and its arity?
   
   If a producer sends over a call to the add function with input types `int32, int32` and output type `int32`, then the consumer would look that up, and if it's able to execute that IR, then it does and if it's not able to do so it returns an error.

##########
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:
       Thought about this over the weekend, and I'm not very keen on renaming everything with a `Wrapper` suffix.
   
   The objects are named the way they are named for a specific reason, and that is to indicate that they are the objects developers of IR producers/consumers should use.
   
   With `Foo` and `FooWrapper`, it's not clear to me as a developer that I should use the `FooWrapper` and not the `Foo`, versus `Foo` and `FooImpl`, which to my eye is much more clear about each object's publicity. I could of course add a comment about that, but that's net additional work versus what's already here, for very little gain other than to remind people.

##########
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 think the purpose of this field is potentially being misunderstood.
   
   There are two very broad use cases that this is designed for:
   
   1. A producer and a consumer that are not self-contained. This is the broadest use case and I think will be the primary way in which the IR is used. In this scenario, the producer inserts a type into this field as the output of its type system. The consumer is then required to adhere to what the producer asked for, or return an error.
   2. A producer and consumer that are self-contained. This is for a system such as a relational database (e.g., DuckDB), where the IR is being used by a system that controls both the producer and the consumer. The purpose of the type field here is to have a place for the output of a type derivation step to be stored after its derivation for later consumption.
   
   Additionally, the field is optional to allow for the type derivation phase to happen at any point in time.
   
   What I think would be a mistake is to not have this field here and assume the producer and consumer will necessarily derive the same output type for every expression. We've already said that defining output type derivation rules are the responsibility of producers. Are we changing that decision?
   
   Regarding performance, I don' t think that's something we should focus on until we're happy with the design that supports the use cases we want to support, and until we have some clarity on what is actually expensive in real-world use cases.

##########
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 think that the rational is inverted: usually planners (IR producers) are the ones that need to know which signatures the consumers accept, so that they can plan eventual detours (e.g. perform some casts to match signatures).
   
   I get what you're saying here, but right now the IR is explicitly not concerned with how (or even whether) a producer is aware of what types of operations are accepted by a consumer.
   
   This is compatible with systems that *do* have additional knowledge; the IR is designed to be unopinionated about how a producer deduces types.
   
   The flow of execution you're describing is compatible with the current IR if I understand the flow correctly: the producer can query the possible operations, figure out how (and whether) to execute the desired operations, and then put all of that into IR and send it to the consumer.
   
   > I would imagine that consumers would have a registry of implemented functions, which they would inform producers about before planning, so that the producer can error out or "dodge via cast" during planning.
   
   Indeed, see the discussion about registries/signatures here https://github.com/apache/arrow/pull/10934/files#r697016776
   
   > The main question for me is whether we want to use logical types or physical types.
   
   You can use either, unless the type systems for representing physical and logical types are distinct trees. AFAICT Arrow doesn't distinguish between logical and physical types in `Schema.fbs`. Is that correct?

##########
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:
       @westonpace 
   
   > This sounds like the user needs to create overloads based solely on the return type. 
   
   Assuming by "user" you mean IR producer, then what you said isn't correct if I understand you correctly. The IR producer's behavior with respect to what IR it chooses to generate is entirely up to the producer. The IR has nothing to do with type derivation at all.
   
   > This feels like a slippery slope. Wouldn't it be better for DuckDB in this case to extend the fbs with their own fields?
   Maybe. We can certainly just leave out types altogether and tell consumers that they need to have a way to determine the type of an expression in IR. To me that seems like it's going to breed a bunch of code duplication across different consumers, but maybe it's less onerous than I think.

##########
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:
       @westonpace 
   
   > This sounds like the user needs to create overloads based solely on the return type. 
   
   Assuming by "user" you mean IR producer, then what you said isn't correct if I understand you correctly. The IR producer's behavior with respect to what IR it chooses to generate is entirely up to the producer. The IR has nothing to do with type derivation at all.
   
   > This feels like a slippery slope. Wouldn't it be better for DuckDB in this case to extend the fbs with their own fields?
   
   Maybe. We can certainly just leave out types altogether and tell consumers that they need to have a way to determine the type of an expression in IR. To me that seems like it's going to breed a bunch of code duplication across different consumers, but maybe it's less onerous than I think.




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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690625806



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       actually, I think creating a struct { ulong padding_for_struct_alignment, int padding_for_byte_align, bytes: byte[] } and making that member should be guaranteed to align.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689841148



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;
+}
+
+/// The contents of Relation.options will be UnionOptions
+/// if Relation.relation_name = "union"
+table UnionOptions {
+  /// For simplicity, all rows from any input to a "union" relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.

Review comment:
       It is, but I'm wondering if there are cases where a union doesn't collapse into a `select distinct *` query.




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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690584181



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+table Call {
+  /// The namespaced name of the function whose invocation this Call represents.
+  /// For example: "arrow::add" or "gandiva::jit_3432".
+  ///
+  /// Names with no namespace are reserved for canonicalization.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  /// For example: "arrow::hash_join" or "gandiva::filter_and_project".
+  ///
+  /// Names with no namespace are reserved for canonical, "pure" relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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);
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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 namespaced name of the join to use. Non-namespaced names are
+  /// reserved for canonicalization. Current names include:
+  ///   "inner"
+  ///   "left"
+  ///   "right"
+  ///   "outer"
+  ///   "cross"
+  join_name: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream

Review comment:
       I don't understand this comment exactly.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712388156



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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,

Review comment:
       Just part of an outdated comment :)




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712417989



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+table Remap {
+  mapping: [uint32] (required);

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712407024



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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

Review comment:
       Done.

##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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,

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712414546



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
wesm commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691335159



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;

Review comment:
       Per Operation below, I might prefer to have the opaque options only in the NonCanonicalFunction table — in the CanonicalFunction, if needed — may not be yet — we could add a union-of-tables providing function-specific options without the need for InlineBuffer




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



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

Posted by GitBox <gi...@apache.org>.
wesm commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691239639



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.operation = CanonicalOperation::Filter
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.operation = CanonicalOperation::Project
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.operation = CanonicalOperation::Aggregate
+table AggregateOptions {
+  /// 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 : uint32 {
+  Inner,
+  LeftOuter,
+  RightOuter,
+  FullOuter,
+  Cross,
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string (required);
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind, NonCanonicalJoinKind
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table JoinOptions {
+  /// 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);
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.operation = CanonicalOperation::OrderBy
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.operation = CanonicalOperation::Limit
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.operation = CanonicalOperation::Common
+table CommonOptions {
+  /// 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 contents of Relation.options will be UnionOptions
+/// if Relation.operation = CanonicalOperation::Union
+table UnionOptions {
+  /// For simplicity, all rows from any input to a union relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.operation = CanonicalOperation::Literal
+table LiteralOptions {
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// 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;
+}
+
+/// Frequently there is an open direct channel between producers
+/// of queries (client) and their consumers (server).
+/// `Interactive` is provided to reify this case:
+rpc_service Interactive {

Review comment:
       I believe anything related to gRPC is out of scope for the specification

##########
File path: cpp/build-support/update-flatbuffers.sh
##########
@@ -23,16 +23,18 @@
 CWD="$(cd "$(dirname "${BASH_SOURCE[0]:-$0}")" && pwd)"
 SOURCE_DIR=$CWD/../src
 FORMAT_DIR=$CWD/../../format
-FLATC="flatc -c --cpp-std c++11"
+FLATC="flatc --cpp --grpc --cpp-std c++11"

Review comment:
       I do not think we should generate gRPC extensions here

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);

Review comment:
       It seems simpler to put the field/output type in `Expression` instead of replicating it across each type in `ExpressionImpl`

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.operation = CanonicalOperation::Filter
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.operation = CanonicalOperation::Project
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.operation = CanonicalOperation::Aggregate
+table AggregateOptions {
+  /// 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 : uint32 {
+  Inner,
+  LeftOuter,
+  RightOuter,
+  FullOuter,
+  Cross,
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string (required);
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind, NonCanonicalJoinKind
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table JoinOptions {
+  /// 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);
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.operation = CanonicalOperation::OrderBy
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.operation = CanonicalOperation::Limit
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.operation = CanonicalOperation::Common
+table CommonOptions {
+  /// 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 contents of Relation.options will be UnionOptions
+/// if Relation.operation = CanonicalOperation::Union
+table UnionOptions {
+  /// For simplicity, all rows from any input to a union relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.operation = CanonicalOperation::Literal
+table LiteralOptions {
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// A specification of a query.
+table Plan {
+  /// One or more output relations.
+  sinks: [Relation] (required);

Review comment:
       What is the interpretation of multiple outputs (versus a single output)? 

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction

Review comment:
       Presumably we would want to add a UserDefinedFunction here which is able to pass an inline buffer containing the serialized function 

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);

Review comment:
       Per below, if you lift the output type (field) into Expression (if we agree that every Expression needs an output Field — name and type), than you can simply call this `ExpressionOp`, since then `Expression` serves more purpose than simply being wrapper for `Expression.impl`

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       I notice the `ForceVectorAlignment` function in C++
   
   https://github.com/google/flatbuffers/blob/master/include/flatbuffers/flatbuffers.h#L1755
   
   We should look at whether
   
   ```
   table InlineBuffer {
     padding_for_alignment:ulong = 0 (required);
     bytes:[ubyte] (required);
   }
   ```
   
   guarantees `bytes` to be aligned

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;

Review comment:
       Having the options as a nested / independent flatbuffer for all canonical operations still makes me squirm. What do you think about putting a union of options for the canonical types in `CanonicalOperation` so that dealing with the InlineBuffer for "built-ins" is not necessary? 

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;

Review comment:
       Per Operation below, I might prefer to have the opaque options only in the NonCanonicalFunction table — in the CanonicalFunction, if needed — may not be yet — we could add a union-of-tables providing function-specific options without the need for InlineBuffer




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697512358



##########
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:
       Yup, will change to `measures`.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698463925



##########
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:
       Thought about this over the weekend, and I'm not very keen on renaming everything with a `Wrapper` suffix.
   
   The objects are named the way they are named for a specific reason, and that is to indicate that they are the objects developers of IR producers/consumers should use.
   
   With `Foo` and `FooWrapper`, it's not clear to me as a developer that I should use the `FooWrapper` and not the `Foo`, versus `Foo` and `FooImpl`, which to my eye is much more clear about each object's publicity. I could of course add a comment about that, but that's net additional work versus what's already here, for very little gain other than to remind people.




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



[GitHub] [arrow] wesm commented on pull request #10934: [RFC] Arrow Compute Serialized Intermediate Representation draft for discussion

Posted by GitBox <gi...@apache.org>.
wesm commented on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-919626099


   Per my comments on the mailing list, if this helps unblock our experimentation with a serialized-query-plan-type interface in C++, and it is abundantly clear that it is experimental (it might be a good idea to steer clear of the format/ directory altogether, somewhere else perhaps?) and aiding with dependent developments, then merging this seems okay with me. 
   
   Hopefully we can work toward convergence with what is being discussed in Substrait (though having to use protobufs fills me with dread — we could use https://github.com/nanopb/nanopb to avoid taking on a libprotobuf dependency that we don't want in libarrow), but if not we can cross that bridge when we get there. 


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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712389054



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.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 = 0;
+}
+
+/// A function call expression
+table Call {
+  /// The function to call
+  name: string (required);
+
+  /// The arguments passed to `name`.
+  arguments: [Expression] (required);
+
+  /// Possible ordering of input. These are useful
+  /// in aggregates where ordering in meaningful such as
+  /// string concatenation
+  orderings: [SortKey];
+}
+
+/// A single WHEN x THEN y fragment.
+table CaseFragment {
+  match: Expression (required);
+  result: Expression (required);
+}
+
+/// Conditional case statement expression
+table ConditionalCase {
+  /// List of conditions to evaluate
+  conditions: [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.
+  else: Expression (required);
+}
+
+/// Switch-style case expression
+table SimpleCase {
+  /// The expression whose value will be matched
+  expression: Expression (required);
+  /// Matches for `expression`
+  matches: [CaseFragment] (required);
+  /// The default value if no cases match
+  else: Expression (required);
+}
+
+/// Whether lesser values should precede greater or vice versa,
+/// also whether nulls should preced or follow values
+enum Collation : uint8 {

Review comment:
       I guess I was going for a word that is a slight generalization of ordering to handle the clustered case, but we can also remove the clustering notion and keeping ordering, until we need the clustering variant.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r699398416



##########
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 think that the rational is inverted: usually planners (IR producers) are the ones that need to know which signatures the consumers accept, so that they can plan eventual detours (e.g. perform some casts to match signatures).
   
   I get what you're saying here, but right now the IR is explicitly not concerned with how (or even whether) a producer is aware of what types of operations are accepted by a consumer.
   
   This is compatible with systems that *do* have additional knowledge; the IR is designed to be unopinionated about how a producer deduces types.
   
   The flow of execution you're describing is compatible with the current IR if I understand the flow correctly: the producer can query the possible operations, figure out how (and whether) to execute the desired operations, and then put all of that into IR and send it to the consumer.
   
   > I would imagine that consumers would have a registry of implemented functions, which they would inform producers about before planning, so that the producer can error out or "dodge via cast" during planning.
   
   Indeed, see the discussion about registries/signatures here https://github.com/apache/arrow/pull/10934/files#r697016776
   
   > The main question for me is whether we want to use logical types or physical types.
   
   You can use either, unless the type systems for representing physical and logical types are distinct trees. AFAICT Arrow doesn't distinguish between logical and physical types in `Schema.fbs`. Is that correct?

##########
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:
       @westonpace 
   
   > This sounds like the user needs to create overloads based solely on the return type. 
   
   Assuming by "user" you mean IR producer, then what you said isn't correct if I understand you correctly. The IR producer's behavior with respect to what IR it chooses to generate is entirely up to the producer. The IR has nothing to do with type derivation at all.
   
   > This feels like a slippery slope. Wouldn't it be better for DuckDB in this case to extend the fbs with their own fields?
   Maybe. We can certainly just leave out types altogether and tell consumers that they need to have a way to determine the type of an expression in IR. To me that seems like it's going to breed a bunch of code duplication across different consumers, but maybe it's less onerous than I think.

##########
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:
       @westonpace 
   
   > This sounds like the user needs to create overloads based solely on the return type. 
   
   Assuming by "user" you mean IR producer, then what you said isn't correct if I understand you correctly. The IR producer's behavior with respect to what IR it chooses to generate is entirely up to the producer. The IR has nothing to do with type derivation at all.
   
   > This feels like a slippery slope. Wouldn't it be better for DuckDB in this case to extend the fbs with their own fields?
   
   Maybe. We can certainly just leave out types altogether and tell consumers that they need to have a way to determine the type of an expression in IR. To me that seems like it's going to breed a bunch of code duplication across different consumers, but maybe it's less onerous than I think.

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

Review comment:
       Done

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

Review comment:
       Done

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

Review comment:
       Fixed.

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

Review comment:
       Done, added a unit.

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

Review comment:
       Added.

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

Review comment:
       Added `DurationLiteral`.




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



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

Posted by GitBox <gi...@apache.org>.
kkraus14 commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698845955



##########
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:
       I figured there'd be some engines that handles NaN ordering differently. I.E. in Pandas with a `float64` dtype column it allows setting `NaN` similarly to what we're allowing for nulls here using an `na_position` argument. This does work as expected when using the experimental `Float64` nullable type though.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691283471



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction

Review comment:
       I'm not sure how that's distinct from providing a NonCanonicalFunction with the serialized function in the options blob




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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r699645607



##########
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:
       > Assuming by "user" you mean IR producer, then what you said isn't correct if I understand you correctly. The IR producer's behavior with respect to what IR it chooses to generate is entirely up to the producer. The IR has nothing to do with type derivation at all.
   
   Correct, but can you give a concrete example of when an IR producer would need to specify the output type?  Why wouldn't a cast be sufficient?




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690586243



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+table Call {
+  /// The namespaced name of the function whose invocation this Call represents.
+  /// For example: "arrow::add" or "gandiva::jit_3432".
+  ///
+  /// Names with no namespace are reserved for canonicalization.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  /// For example: "arrow::hash_join" or "gandiva::filter_and_project".
+  ///
+  /// Names with no namespace are reserved for canonical, "pure" relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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);
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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 namespaced name of the join to use. Non-namespaced names are
+  /// reserved for canonicalization. Current names include:
+  ///   "inner"
+  ///   "left"
+  ///   "right"
+  ///   "outer"
+  ///   "cross"
+  join_name: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream

Review comment:
       I'll expand the comment




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689824241



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;

Review comment:
       This should indeed be required.
   ```suggestion
     name: string (required);
   ```




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689754664



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape

Review comment:
       Tiny nit: do we need to repeat the `Shape` suffix?

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.

Review comment:
       Perhaps a short comment about the potentially non-obvious case of `struct`s here might be useful to indicate the full scope of what `Scalar` entails. I can also push up a blurb.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;
+}
+
+/// The contents of Relation.options will be UnionOptions
+/// if Relation.relation_name = "union"
+table UnionOptions {
+  /// For simplicity, all rows from any input to a "union" relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.

Review comment:
       I'm not entirely sure we can leave out something like a `unique: boolean;` parameter, since chaining unions with different value of such a parameter is (syntactically) legal:
   
   ```sql
   SELECT 1
   UNION ALL
   SELECT 1
   UNION
   SELECT 1
   ```
   
   Maybe there's an algebraic equivalence here that obviates the need for such a parameter, but I don't see it.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;
+}
+
+/// The contents of Relation.options will be UnionOptions
+/// if Relation.relation_name = "union"
+table UnionOptions {
+  /// For simplicity, all rows from any input to a "union" relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.relation_name = "literal"
+table LiteralOptions {
+  /// Batches of rows in this literal.
+  batches: [RecordBatch] (required);
+}
+
+/// A specification of a query.
+table Plan {
+  /// One or more output relations.
+  sinks: [Relation] (required);
+
+  /// Dictionary batches which may be referenced by Literals.
+  dictionary_batch: [DictionaryBatch];

Review comment:
       Can you elaborate a tiny bit on why dictionary batches are in `Plan` and not in `LiteralOptions`?

##########
File path: docs/source/format/ComputeIR.rst
##########
@@ -0,0 +1,59 @@
+.. 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.
+
+**********************************************
+Arrow Compute IR (Intermediate Representation)
+**********************************************
+
+In the same way that the Arrow format provides a powerful tool
+for communicating data, Compute IR is intended to provide a
+consistent format for representing analytical operations against
+that data. As an arrow-native expression of computation it includes
+information such as explicit types and schemas and arrow formatted
+literal data. It is also optimized for low runtime overhead in both
+serialization and deserialization.
+
+Built-in definitions are included to enable representation of
+relational algebraic operations- the contents of a "logical query plan".
+Compute IR also has first class support for representing operations
+which are not members of a minimal relational algebra, including
+implementation and optimization details- the contents of a "physical
+query plan". This approach is taken in emulation of `MLIR`_ (Multi-Level
+Intermediate Representation), a system which has had strong successes in
+spaces of comparable complexity to representation of analytic operations.
+To borrow terms from that project, there are two mutations of interest:
+
+* Replacement of representations with semantically equivalent representations
+  which will yield better performance for consumers- an optimization pass.
+* Replacement of abstract or generic representations with more specific
+  and potentially consumer-specific representations- a lowering pass.
+  This modification corresponds to the translation of a logical plan
+  to a physical plan.
+
+Allowing representation of physical plans (and plans which are between
+logical and physical) in Compute IR enables systems to define incremental
+optimization and lowering passes which operate on and produce valid
+Compute IR. This in turn enables communication, manipulation, and inspection
+at every stage of lowering/optimization by the same tools
+used for logical-plan-equivalent-IR. This is especially useful for systems
+where such passes may depend on information only available on every node
+of a distributed consumer (for example statistics unique to that node's
+local data) or may not be universal to all backends in a heterogeneous
+consumer (for example which optimizations nodes are capable of for
+non equi joins).

Review comment:
       Great summary.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;

Review comment:
       Are `Buffer`s meaningfully different from `[ubyte]`?

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;

Review comment:
       Should this be `(required)`? What does  it mean for a CTE or alias to lack a name?

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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:
       What was the rationale for combining these into one enum?

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;

Review comment:
       :( People are gonna find ways to invent new names for joins that we won't think of.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];

Review comment:
       Should this be required? There's no meaningful semantic difference between a missing `keys` and an empty `keys` as far as I can tell.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697041056



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction

Review comment:
       Per my comments elsewhere, if we want to support serialized functions, we should make them structured, not just a bucket of bytes. For example:
   ```
   {
     type: "python_pickle", 
     argument_types: [int], 
     output_type:"int", 
     dynamic:"false", 
     maintains: [sort, cluster, distribution],  
     python_prerequisites: [pyarrow, pytorch],
     bytes:<bytes>"
   }
   ```
   
   The structure should be such that: 
   - tools don't have to know how to decode the bytes to do plan transformations
   - transformations can move operations without having to worry about invalidating correctness (for example move where distribution of data is happening in a plan)
   - the plan can be validated without doing a byte decoding




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



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

Posted by GitBox <gi...@apache.org>.
kkraus14 commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697654000



##########
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:
       I think we also need the ability to control `NaN` ordering here in the case of floating point columns. I.E. some systems treat `NaN` as larger than all other values but smaller than `null`, others do differently.




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



[GitHub] [arrow] github-actions[bot] removed a comment on pull request #10934: [RFC] Arrow Compute Serialized Intermediate Representation draft for discussion

Posted by GitBox <gi...@apache.org>.
github-actions[bot] removed a comment on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-898655955


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/master/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on JIRA? https://issues.apache.org/jira/browse/ARROW
   
   Opening JIRAs ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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



[GitHub] [arrow] cpcloud closed pull request #10934: ARROW-14062: [Format] Initial arrow-internal specification of compute IR

Posted by GitBox <gi...@apache.org>.
cpcloud closed pull request #10934:
URL: https://github.com/apache/arrow/pull/10934


   


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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712390903



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+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 {
+  /// Output remapping of ordinal columns for a given operation
+  output_mapping: Emit (required);
+}
+
+/// Filter operation
+table Filter {
+  /// Common options
+  base: RelBase (required);
+  /// Child relation
+  rel: Relation (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 {

Review comment:
       If it turns out to simplify the producer/consumer code to roll these into `Call`s then we can certainly make that change and refactor.




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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690623607



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       I'm not sure Unions in FB are equivelant to union's in C++ (I don't know that if you select a byte union, you end up sharing the same memory).




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697565133



##########
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 think it is important for the IR to have a clear definition of what must be headed. Having critical functional information in a custom metadata field feels counter to that goal.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r699557968



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

Review comment:
       Done, added a unit.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712399612



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  values: [Literal];
+}
+
+table StructLiteral {
+  values: [KeyValue];
+}
+
+table KeyValue {
+  key: string;
+  value: Literal;
+}
+
+table MapLiteral {
+  values: [KeyValue];

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697551237



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

Review comment:
       > 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?
   
   Good point. Will remove that.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688834402



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}

Review comment:
       Again, I'd defer this configuration to a function call rather than introduce more options




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691396153



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);

Review comment:
       I agree that every Expression needs type, but I disagree that name is meaningful for Expressions. To my mind a name is ascribed to the referent by the referring entity; to speak in graph theory it's an edge property rather than a node property. To give a practical context: in a projection like `SELECT $complicated_expr as A, $complicated_expr as B` it should not be an error to use a single memoized instance of `$complicated_expr`- but including name as an expression property would make these two incompatible.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712386608



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;

Review comment:
       Good call, I had forgotten about negative scale. Been a while since I've `Decimal`d :)




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



[GitHub] [arrow] cpcloud commented on pull request #10934: [RFC] Arrow Compute Serialized Intermediate Representation draft for discussion

Posted by GitBox <gi...@apache.org>.
cpcloud commented on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-920305415


   Ok, I'm going to move this code to a top-level `experimental/computeir` directory, and remove the generated Python code for now. Thanks all for the review.


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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688831853



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}

Review comment:
       Maybe it'd be best to provide a single enum: `ASCENDING_THEN_NULLS, NULLS_THEN_ASCENDING, ...`




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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690625806



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       actually, I think creating a struct { int padding, bytes: byte[] } and making that member should be guaranteed to align.




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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690580363



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       this has implications for languages that aren't C++ (I don't think the ergonmics of converting long[] to byte[] is quite as easy in ava.




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



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

Posted by GitBox <gi...@apache.org>.
kkraus14 commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698845955



##########
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:
       I figured there'd be some engines that handles NaN ordering differently. I.E. in Pandas with a `float64` dtype column it allows setting `NaN` similarly to what we're allowing for nulls here using an `na_position` argument. This does work as expected when using the experimental `Float64` nullable type though.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697038680



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.operation = CanonicalOperation::Filter
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.operation = CanonicalOperation::Project
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.operation = CanonicalOperation::Aggregate
+table AggregateOptions {
+  /// 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 : uint32 {
+  Inner,
+  LeftOuter,
+  RightOuter,
+  FullOuter,
+  Cross,
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string (required);
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind, NonCanonicalJoinKind
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table JoinOptions {
+  /// 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);
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.operation = CanonicalOperation::OrderBy
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.operation = CanonicalOperation::Limit
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.operation = CanonicalOperation::Common
+table CommonOptions {
+  /// 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 contents of Relation.options will be UnionOptions
+/// if Relation.operation = CanonicalOperation::Union
+table UnionOptions {
+  /// For simplicity, all rows from any input to a union relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.operation = CanonicalOperation::Literal
+table LiteralOptions {
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// A specification of a query.
+table Plan {
+  /// One or more output relations.
+  sinks: [Relation] (required);

Review comment:
       In general, this feels like it is beyond the scope of a plan and is more about a specific read or write relation as well as communication of plan between systems. Let's try to avoid baking those concepts into the core plan.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712397183



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  values: [Literal];
+}
+
+table StructLiteral {
+  values: [KeyValue];
+}
+
+table KeyValue {
+  key: string;
+  value: Literal;
+}
+
+table MapLiteral {
+  values: [KeyValue];

Review comment:
       It was, but only because that's the most common case, and we wanted to keep the prototype fairly limited. However, I think we can allow any type here if Arrow allows it.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-919628055


   Don't fill with dread @wesm! For substrait we're more interested in the spec than the fornat. One goal is to have several serialization formats supported. Additionally, no decisions have been made for serialization in general. Protobuf may be in the sketch but the sketch is only designed to drive towards consensus, not constrain the possible outcome.


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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690466642



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.

Review comment:
       I had intended to use `::`, I'll clarify with examples




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697583100



##########
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:
       It seems then that the only way to do that would be to extend `Type` which is definitely out of scope for this PR :)




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



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

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698856769



##########
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 think that the rational is inverted: usually planners (IR producers) are the ones that need to know which signatures the consumers accept, so that they can plan eventual detours (e.g. perform some casts to match signatures).
   
   For example, if a consumer does not support `(add,Date32,Duration(seconds))`, there are ways around this via casting and arithmetics; the planner may want to get around this limitation.
   
   Some examples where typing is needed:
   
   ```
   sum(f32) -> (f32 or f64)?
   sum(i32) -> (i32 or i64)?
   add(i32,i32) -> (i32 or i64)?
   ```
   
   different code will need to run depending on the requested output type. If an IR consumer does not support some of the calls, the producer needs to know that so that it can emit a different set of operators (e.g. perform some casts before and after the main operator).
   
   `DataFusion` dynamic dispatch is based on this idea; the physical planner plans out which physical expressions will be used based on the input schema. Broadly, it goes like this:
   
   1. every operator has an ordered set of valid input types that it supports (e.g. `add` could be `(i32,i32),(i64,i64),(f32,f32),(f64,f64)` in this order; ordering is from least informative (in Shannon's notion) to most informative.
   2. for every operator, there is a map between input and output: `output_type(op,arg_types) -> arg_type` (e.g. `output_type("add", [i32,i32]) -> i64`)
   3. the first lossless coercion to fit the signature is selected (e.g. if input types are `(i32,i64)`, they are mapped to `(i64,i64)` prior to being passed to `add` (since the `(i32,i32)` would be lossy)).
   
   Given an input schema (vector of of `DataType`), all IR calls are known in advance via the type resolution of the physical nodes available + applied coercion.  `DataFusion`'s planner knows in advance which physical nodes are implemented and uses this information to plan out the operations (since it is in the same source code). UDFs and UDAFs in DataFusion must register this information; in fact, UDFs and UDAFs use the same signatures as builtin functions; the only way they are special is that their implementation is not known to DataFusion and thus can't be inlined (which is fine).
   
   I would imagine that consumers would have a registry of implemented functions, which they would inform producers about _before planning_, so that the producer can error out or "dodge via cast" _during planning_. The producer would use the registry to decide which physical plan it would pass down to the consumer, pretty much like an architecture that declares the registry in their user manual and that compilers use.
   
   The main question for me is whether we want to use logical types or physical types. E.g. is it relevant that a consumer does not implement `add(date32,duration(seconds))` if it implements `add(int32,int64)`? My understanding is that it should not matter since logical semantics can be handled by an IR planner.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697610672



##########
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:
       Agreed, though I don't think IR is that useful without `Read`. I'm not sure if there's actually a common denominator here.
   
   I will look around at the different ways this has been done. So far, it looks like DuckDB implements different sources as table producing functions, maybe that's a good place to start generalizing from.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r699557711



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

Review comment:
       Done

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

Review comment:
       Done

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

Review comment:
       Fixed.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689826684



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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:
       It seems clearer as a single enum, based on discussion https://github.com/apache/arrow/pull/10934#discussion_r688802634




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689826989



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];

Review comment:
       I agree

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];

Review comment:
       ```suggestion
     keys: [Expression] (required);
   ```




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697591569



##########
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:
       An alternative is to simply remove cast as a first class operation. That way it can simply be one or more function definitions. Some can include length and others will not need them.




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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691373528



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;

Review comment:
       Another pattern might be to make CanonicalOperationId "CanonicalOperation" which is a union of the options.  It doesn't fully get out of slightly harder pattern matching code, but it would eliminate a separate discrimant.




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



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

Posted by GitBox <gi...@apache.org>.
wesm commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697746512



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

Review comment:
       nit: the Arrow spec has left the door open to smaller decimals

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

Review comment:
       nit: need unit 

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

Review comment:
       nit: should we put a unit here? 

##########
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.
   
   I think for scalar functions, since there are so many (but not necessarily relational operators), I also prefer this approach to having all the functions listed in an enum on the Flatbuffers file, so that we can have an append-only yaml file with all the functions. Using an integer id to identify a function versus a string makes the IR smaller (good — only ever need 4 bytes, even only 2 bytes, to identify a function) and implementations marginally less complicated since many engines will have string identifiers for functions that are different than the "canonical" names in our function inventory. The inventory of scalar functions is likely to grow very fast and not having to modify the Flatbuffers files would be beneficial 
   
   > 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.
   
   I agree with this
   
   > Each of these will actually have many variations and each should be identified separately add(int,int) => int vs add(bigint,bigint) => bigint.
   
   I see arguments both ways on using a different function id for each overload of a function. The main annoyance I would see in having a different id for each overload of a function is that IR implementations would have to maintain a huge IR mapping table, versus using a common id for all variants of "add" for example. On the other hand, if there is a different id for every overload, then it leaves no ambiguity about what the input and output types should be. But that is a massive scope increase for this project to enumerate all the function signatures of every function overload contemplated... 

##########
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 think it would be useful to be able to serialize and send e.g. `sum($expr_0) / mean($expr_1)` (with these expressions being possibly unbound to a particular table schema) without having to build an aggregation relational operator — if aggregation function calls are "different" then the type system to achieve this is probably a bit more complex, if you have ideas

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

Review comment:
       nit: ListLiteral ?

##########
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:
       I agree that whatever decision is made with the scalar functions should be consistent here

##########
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:
       I agree where a table is being introduced solely to work around Flatbuffers' union issues, that calling it `Wrapper` would make it more clear. The rule then would be to never use a `ThingWrapper` as a member of any other table, only where you need to put the wrapped union in an array or serialize it as a top-level Flatbuffers object 

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

Review comment:
       nit: need Duration

##########
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 think this is an important question — connected to the above question about enumerating function signatures — and should be raised and discussed more broadly on the mailing list and try to include the larger group of people who commented in the Google Doc that I made originally. 
   
   For built-in / canonical functions where we expect `$func($type_0, ...)` to yield a deterministic output type, there isn't much motivation to serialize the output type — you would only want to put the output type there when it is adding useful information. If you always put it there, you're paying the cost of serializing and deserializing a Field for every expression. An IR producer could run in "verbose" mode and put all the output types (if it were useful to a IR consumer that doesn't have the type derivation logic)
   
   I suspect that there will be a need to build an inventory of function signatures to reduce ambiguity and to make things more straightforward for IR implementations (for example, an implementation could read the input/output type rules from a text file — or generate code — rather than having to enter the type derivations by hand)




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



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

Posted by GitBox <gi...@apache.org>.
wesm commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691333084



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;

Review comment:
       Having the options as a nested / independent flatbuffer for all canonical operations still makes me squirm. What do you think about putting a union of options for the canonical types in `CanonicalOperation` so that dealing with the InlineBuffer for "built-ins" is not necessary? 




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712395749



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {
+  SECOND,
+  MILLISECOND,
+  MICROSECOND,
+  NANOSECOND,
+}
+
+table TimeLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+table TimestampLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit;
+
+  // 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 DurationLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+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.

Review comment:
       I'll add it.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688739400



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+table Literal {
+  /// Number of slots. If this is absent, the literal should be
+  /// considered scalar.
+  length: long;
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+enum JoinKind : int {
+  INNER,
+  LEFT,
+  RIGHT,
+  FULL,
+  SEMI,
+  ANTI,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: JoinKind;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.name = "literal"
+table LiteralOptions {
+  /// Batches of rows in this literal.
+  batches: [RecordBatch] (required);
+  /// A dictionary batch for this literal.
+  dictionary_batch: DictionaryBatch;
+}
+
+/// The contents of Relation.options will be InteractiveOutputOptions
+/// if Relation.name = "interactive_output"
+table InteractiveOutputOptions {
+  /// In an interactive context there is an implicit pipe open between
+  /// the producer and the consumer along which output may be sent.

Review comment:
       Terminating a plan with a node like this allows interactive contexts to emulate the behavior of SQL `SELECT * FROM tab;` where the query evaluates to a table.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688741589



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,211 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+table Literal {
+  /// Number of slots. If this is absent, the literal should be
+  /// considered scalar.
+  length: long;
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+enum JoinKind : int {
+  INNER,
+  LEFT,
+  RIGHT,
+  FULL,
+  SEMI,
+  ANTI,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: JoinKind;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.name = "literal"
+table LiteralOptions {
+  /// Batches of rows in this literal.
+  batches: [RecordBatch] (required);
+  /// A dictionary batch for this literal.
+  dictionary_batch: DictionaryBatch;
+}
+
+/// The contents of Relation.options will be InteractiveOutputOptions
+/// if Relation.name = "interactive_output"
+table InteractiveOutputOptions {
+  /// In an interactive context there is an implicit pipe open between
+  /// the producer and the consumer along which output may be sent.
+  /// `id` is an opaque identifier for such outputs; for example it could be used to
+  /// identify which Jupyter output cell should receive the output.
+  id: long;

Review comment:
       ```suggestion
     options: Blob;
   ```




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689831577



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;
+}
+
+/// The contents of Relation.options will be UnionOptions
+/// if Relation.relation_name = "union"
+table UnionOptions {
+  /// For simplicity, all rows from any input to a "union" relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.

Review comment:
       Wouldn't that be equivalent to
   ```sql
   with s as (
     select 1
     union all
     select 1
     union all
     select 1
   )
   select distinct * from s;
   ```
   ?
   




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698458633



##########
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:
       Are we actually sure this is possible to do for all of the types we want to support?
   
   What the key is for, say, the `==` operator for a complex type like list or struct?
   
   I don't think a wildcard type is well-defined here without more clarification. For example, `List<T>` can only be compared with `List<U>` if `T == U`, but if `T != U` the operation is undefined.
   
   Unnest is another example.
   
   With a type system that handles generics, you can't write down the type of all possible instantiations of any type that has a type parameter, such as list, map, and struct.
   
   What is the issue with having a list of functions in some structured format, that indicates the canonical name of the function and its arity?
   
   If a producer sends over a call to the add function with input types `int32, int32` and output type `int32`, then the consumer would look that up, and if it's able to execute that IR, then it does and if it's not able to do so it returns an error.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697617635



##########
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:
       That seems like a it will be just as hard to maintain as if we append to an enum.
   
   > Each of these will actually have many variations and each should be identified separately add(int,int) => int vs add(bigint,bigint) => bigint.
   
   I don't follow why operations with multiple overloads need to be dealt with at all in the IR. Wouldn't a function have a singular definition (or be singularly derivable) for a given IR?
   
   Other than performance concerns about passing lots of strings around, what's the problem with a name and an optional namespace string for _every_ function?




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697518715



##########
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:
       Makes sense, adding it.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697511527



##########
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:
       Ah, you can't do that, because of the type recursion :(




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



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

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690442884



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.

Review comment:
       What is the namespace separator (`.`, `::`, ?)

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Buffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream
+  /// of data and reusing it, potentially multiple times and
+  /// potentially recursively.
+  name: string;
+}
+
+/// The contents of Relation.options will be UnionOptions
+/// if Relation.relation_name = "union"
+table UnionOptions {
+  /// For simplicity, all rows from any input to a "union" relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.

Review comment:
       You can always rewrite a single `select … UNION select…` as `select distinct * from (select … UNION ALL select …)` so I would think they're always equivalent.




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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690580363



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       this has implications for languages that aren't C++ (I don't think the ergonmics of converting long[] to byte[] is quite as easy in ava.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);

Review comment:
       what this indirection?

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+table Call {
+  /// The namespaced name of the function whose invocation this Call represents.
+  /// For example: "arrow::add" or "gandiva::jit_3432".
+  ///
+  /// Names with no namespace are reserved for canonicalization.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  /// For example: "arrow::hash_join" or "gandiva::filter_and_project".
+  ///
+  /// Names with no namespace are reserved for canonical, "pure" relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "common"
+  ///   "union"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.relation_name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.relation_name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.relation_name = "aggregate"
+table AggregateOptions {
+  /// 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);
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.relation_name = "join"
+table JoinOptions {
+  /// 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 namespaced name of the join to use. Non-namespaced names are
+  /// reserved for canonicalization. Current names include:
+  ///   "inner"
+  ///   "left"
+  ///   "right"
+  ///   "outer"
+  ///   "cross"
+  join_name: string;
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.relation_name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.relation_name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.relation_name = "common"
+table CommonOptions {
+  /// Commons (CTEs in SQL) allow assigning a name to a stream

Review comment:
       I don't understand this comment exactly.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       I'm not sure Unions in FB are equivelant to union's in C++ (I don't know that if you select a byte union, you end up sharing the same memory).

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       you might be able to guarantee alignment by having a required long followed by a bytes: [byte] field

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       actually, I think creating a struct { int padding, bytes: byte[] } and making that member should be guaranteed to align.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       actually, I think creating a struct { ulong padding_for_struct_alignment, int padding_for_byte_align, bytes: byte[] } and making that member should be guaranteed to align.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;

Review comment:
       Another pattern might be to make CanonicalOperationId "CanonicalOperation" which is a union of the options.  It doesn't fully get out of slightly harder pattern matching code, but it would eliminate a separate discrimant.




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



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

Posted by GitBox <gi...@apache.org>.
jacques-n commented on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-907719679


   >> I think it would be useful to be able to serialize and send e.g. sum($expr_0) / mean($expr_1) (with these expressions being possibly unbound to a particular table schema) without having to build an aggregation relational operator
   
   I'm assuming you mean that you want to avoid having to build a project on top of an aggregate? (please confirm my interprestation of what you said).
   
   My initial intuition is that supporting arbitrary expressions in aggregation creates more complexity (and heavier requirement on semantic analysis to confirm plan validity). I agree that there are situations where you might want a compound relational operation that does an aggregation calculation followed immediately by a non-aggregate calculation (e.g. the division in your expression). However, I think that "compound aggregate" would be an additional relational operator we could introduce at a later stage as opposed to being one of the initial primitives (or possibly even be an internal concern/optimization of a particular execution engine).


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



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

Posted by GitBox <gi...@apache.org>.
westonpace commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r699645607



##########
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:
       > Assuming by "user" you mean IR producer, then what you said isn't correct if I understand you correctly. The IR producer's behavior with respect to what IR it chooses to generate is entirely up to the producer. The IR has nothing to do with type derivation at all.
   
   Correct, but can you give a concrete example of when an IR producer would need to specify the output type?  Why wouldn't a cast be sufficient?




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



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

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690624487



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       you might be able to guarantee alignment by having a required long followed by a bytes: [byte] field




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690615060



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       I think this can be resolved by taking a leaf out of parquet2's book and defining InlineBuffer as a union of vectors of each primitive type, I'll try that.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697550246



##########
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:
       Done. Thanks.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690584137



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);

Review comment:
       The generators for some languages don't support vector-of-unions. I'll add a comment explaining that




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688992634



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}

Review comment:
       Note that this isn't necessarily an equality join- any pair of rows for which `on_expression` evaluates to true will be emitted. So here again the relationship between NaNs would be embedded in an expression




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688740240



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+table Literal {
+  /// Number of slots. If this is absent, the literal should be
+  /// considered scalar.
+  length: long;
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+enum JoinKind : int {
+  INNER,
+  LEFT,
+  RIGHT,
+  FULL,
+  SEMI,
+  ANTI,
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: JoinKind;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.name = "order_by"
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.name = "limit"
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.name = "literal"
+table LiteralOptions {
+  /// Batches of rows in this literal.
+  batches: [RecordBatch] (required);
+  /// A dictionary batch for this literal.
+  dictionary_batch: DictionaryBatch;
+}
+
+/// The contents of Relation.options will be InteractiveOutputOptions
+/// if Relation.name = "interactive_output"
+table InteractiveOutputOptions {
+  /// In an interactive context there is an implicit pipe open between
+  /// the producer and the consumer along which output may be sent.
+  /// `id` is an opaque identifier for such outputs.

Review comment:
       ```suggestion
     /// `id` is an opaque identifier for such outputs; for example it could be used to
     /// identify which Jupyter output cell should receive the output.
   ```




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712402103



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712392662



##########
File path: experimental/computeir/Relation.fbs
##########
@@ -0,0 +1,209 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+include "Expression.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// A data type indicating that a different mapping of columns
+/// should occur in the output.
+///
+/// For example:
+///
+/// Given a query `SELECT b, a FROM t` where `t` has columns a, b, c
+/// the mapping value for the projection would equal [1, 0].
+table Remap {
+  mapping: [uint32] (required);

Review comment:
       Names aren't well defined for arbitrary column expressions in projections, right? What is the name of the expression `pow(a + b, 2) * 3` in
   
   ```sql
   SELECT pow(a + b, 2) * 3
   FROM t
   ```
   ?




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712403470



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {

Review comment:
       Done.

##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {
+  SECOND,
+  MILLISECOND,
+  MICROSECOND,
+  NANOSECOND,
+}
+
+table TimeLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit = MILLISECOND;
+}
+
+table TimestampLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: TimeUnit;
+
+  // timezone value, same definition as used in Schema.fbs.
+  timezone: string;
+}
+
+table IntervalLiteralMonths {
+  months: int32;
+}
+
+table IntervalLiteralDaysMilliseconds {
+  days: int32;
+  milliseconds: int32;
+}

Review comment:
       Done.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712387040



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];
+  scale: uint8;
+  precision: uint8;
+}
+
+table BooleanLiteral {
+  value: bool;
+}
+
+table NullLiteral {}
+
+enum DateUnit : uint8 {
+  DAY,
+  MILLISECOND,
+}
+
+table DateLiteral {
+  value: int64;
+
+  // unit of `value`
+  unit: DateUnit = MILLISECOND;
+}
+
+enum TimeUnit : uint8 {

Review comment:
       For some reason it didn't dawn on me to reuse that, I'm not sure why. Thanks.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712395590



##########
File path: experimental/computeir/Literal.fbs
##########
@@ -0,0 +1,203 @@
+// 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 ListLiteral {
+  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: [byte];

Review comment:
       Good call. Yes, this should be little endian encoded. I will add some commentary to that effect.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r697548288



##########
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:
       The idea here is that a producer/consumer can insert the type when it needs to, so that the information is available in the IR for the next phase of processing.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r712411973



##########
File path: experimental/computeir/Expression.fbs
##########
@@ -0,0 +1,212 @@
+// 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 "../../format/Schema.fbs";
+include "Literal.fbs";
+
+namespace org.apache.arrow.computeir.flatbuf;
+
+/// Access a value for a given map key
+table MapKey {
+  key: string (required);

Review comment:
       Ah, it was Literal that needed auditing.




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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r690711297



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,

Review comment:
       We can haggle on the details later but I don't think quantile is well supported enough to be a canonical function.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,368 @@
+// 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.flatbuf.computeir;

Review comment:
       Do we need the `flatbuf` namespace here? Anything consuming this will be using flatbuffers, so putting that in the namespace seems like it just increases the number of characters someone has to type.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction

Review comment:
       `NonCanonicalFunction` and `UserDefinedFunction` are synonymous here. `NonCanonicalFunction` I think is probably a better name as it leaves no room for ambiguity.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,368 @@
+// 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.flatbuf.computeir;
+
+table Int8Buffer { items: [uint8] (required); }
+table Int16Buffer { items: [uint16] (required); }
+table Int32Buffer { items: [uint32] (required); }
+table Int64Buffer { items: [uint64] (required); }
+table UInt8Buffer { items: [uint8] (required); }
+table UInt16Buffer { items: [uint16] (required); }
+table UInt32Buffer { items: [uint32] (required); }
+table UInt64Buffer { items: [uint64] (required); }
+table Float32Buffer { items: [float32] (required); }
+table Float64Buffer { items: [float64] (required); }
+table TableBuffer { items: [ubyte] (required, nested_flatbuffer: "InlineBuffer"); }
+union InlineBufferImpl {
+  Int8Buffer,
+  Int16Buffer,
+  Int32Buffer,
+  Int64Buffer,
+  UInt8Buffer,
+  UInt16Buffer,
+  UInt32Buffer,
+  UInt64Buffer,
+  Float32Buffer,
+  Float64Buffer,
+  TableBuffer,
+}
+
+/// An inline replacement for org.apache.arrow.Buffer because that
+/// requires a sidecar block of bytes into which offsets can point.
+/// A union of buffers of each primitive type is provided to avoid
+/// the need for reinterpret_cast, std::mem::transmute, ...
+/// The final member of the union is a bytes buffer aligned suitably
+/// to hold any flatbuffer Table.
+table InlineBuffer {
+  // Ideally we'd simply have `union InlineBuffer { ... }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: InlineBufferImpl (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  /// Field.name should always be absent since it is not meaningful
+  /// for Expressions.
+  field: Field (required);
+
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,

Review comment:
       Do we need a `GetItem` call here for indexing arrays?

##########
File path: cpp/build-support/update-flatbuffers.sh
##########
@@ -23,16 +23,18 @@
 CWD="$(cd "$(dirname "${BASH_SOURCE[0]:-$0}")" && pwd)"
 SOURCE_DIR=$CWD/../src
 FORMAT_DIR=$CWD/../../format
-FLATC="flatc -c --cpp-std c++11"
+FLATC="flatc --cpp --grpc --cpp-std c++11"

Review comment:
       I agree with that. The example service is a good extension, but I don't think it should be part of the core IR right now.

##########
File path: cpp/build-support/update-flatbuffers.sh
##########
@@ -23,16 +23,18 @@
 CWD="$(cd "$(dirname "${BASH_SOURCE[0]:-$0}")" && pwd)"
 SOURCE_DIR=$CWD/../src
 FORMAT_DIR=$CWD/../../format
-FLATC="flatc -c --cpp-std c++11"
+FLATC="flatc --cpp --grpc --cpp-std c++11"

Review comment:
       One practical reason is that some important languages (such as Python) cannot generate code for services which will cause flatc to fail.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.operation = CanonicalOperation::Filter
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.operation = CanonicalOperation::Project
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.operation = CanonicalOperation::Aggregate
+table AggregateOptions {
+  /// 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 : uint32 {
+  Inner,
+  LeftOuter,
+  RightOuter,
+  FullOuter,
+  Cross,
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string (required);
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind, NonCanonicalJoinKind
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table JoinOptions {
+  /// 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);
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.operation = CanonicalOperation::OrderBy
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.operation = CanonicalOperation::Limit
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.operation = CanonicalOperation::Common
+table CommonOptions {
+  /// 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 contents of Relation.options will be UnionOptions
+/// if Relation.operation = CanonicalOperation::Union
+table UnionOptions {
+  /// For simplicity, all rows from any input to a union relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.operation = CanonicalOperation::Literal
+table LiteralOptions {
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// 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;
+}
+
+/// Frequently there is an open direct channel between producers
+/// of queries (client) and their consumers (server).
+/// `Interactive` is provided to reify this case:
+rpc_service Interactive {

Review comment:
       Yup, this can go in examples to help clarify how `Plan`s can be used.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;

Review comment:
       Since we don't yet have a very specific idea of how people are going to use this, I think we should leave this as InlineBuffers.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,

Review comment:
       I think for now maybe we leave out everything except the bare bones functions that we need.

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;

Review comment:
       I think there's a tradeoff here. Having two code paths for handling canonical versus non canonical increases complexity, since a consumer now has to handle canonical things in additional to still handling InlineBuffer. If we're going to allow InlineBuffer, I don't think there's a good reason not to have just a single way to deal with options.




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



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

Posted by GitBox <gi...@apache.org>.
kkraus14 commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689142166



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;

Review comment:
       That unfortunately doesn't solve the problem. I believe there's use cases out there that need to control specific ordering with regards to `NaN` vs `null` vs other values. I.E. you could imagine all of the following being a desired output for an ascending sort:
   - `[NaN, null, 1.0, 2.0, 5.0]`
   - `[null, NaN, 1.0, 2.0, 5.0]`
   - `[NaN, 1.0, 2.0, 5.0, null]`
   - `[null, 1.0, 2.0, 5.0, NaN]`
   - `[NaN, 1.0, 2.0, 5.0, null]`
   - `[1.0, 2.0, 5.0, NaN, null]`
   - `[1.0, 2.0, 5.0, null, NaN]`
   
   I think to accomplish this you need something like `null_ordering`, `nan_ordering`, and `nan_vs_null` parameters.




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



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

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r688834200



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,210 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// Wrapper for blobs of arbitrary bytes
+table Blob {
+  bytes: [ubyte];
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union Expression {
+  Literal, FieldRef, Call
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Blob;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  type: Type;
+}
+
+/// A relation is a set of rows with consitent schema.
+table Relation {
+  /// The namespaced name of this Relation.
+  ///
+  /// Names with no namespace are reserved for pure relational
+  /// algebraic operations, which currently include:
+  ///   "filter"
+  ///   "project"
+  ///   "aggregate"
+  ///   "join"
+  ///   "order_by"
+  ///   "limit"
+  ///   "literal"
+  ///   "interactive_output"
+  relation_name: string (required);
+
+  /// Parameters for `relation_name`; content/format may be unique to each
+  /// value of `relation_name`.
+  options: Blob;
+
+  /// The arguments passed to `relation_name`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.name = "filter"
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// "filter" relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.name = "project"
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the "project" relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.name = "aggregate"
+table AggregateOptions {
+  /// 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];
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.name = "join"
+table JoinOptions {
+  /// 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);
+  join_kind: string;
+}
+
+/// Whether lesser values should precede greater or vice versa.
+enum Ordering : uint8 {
+  ASCENDING,
+  DESCENDING,
+}
+
+/// Whether nulls should precede or follow other values.
+enum NullOrdering : uint8 {
+  FIRST,
+  LAST
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING;
+  null_ordering: NullOrdering = LAST;

Review comment:
       In general, I'd defer nan coercion to an Expression. In this instance, if `value` may have nans then it should for example be wrapped in a call to the "replace_nans_with_nulls" function to establish a consistent ordering.




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



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

Posted by GitBox <gi...@apache.org>.
wesm commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r691239639



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.operation = CanonicalOperation::Filter
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.operation = CanonicalOperation::Project
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.operation = CanonicalOperation::Aggregate
+table AggregateOptions {
+  /// 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 : uint32 {
+  Inner,
+  LeftOuter,
+  RightOuter,
+  FullOuter,
+  Cross,
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string (required);
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind, NonCanonicalJoinKind
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table JoinOptions {
+  /// 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);
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.operation = CanonicalOperation::OrderBy
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.operation = CanonicalOperation::Limit
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.operation = CanonicalOperation::Common
+table CommonOptions {
+  /// 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 contents of Relation.options will be UnionOptions
+/// if Relation.operation = CanonicalOperation::Union
+table UnionOptions {
+  /// For simplicity, all rows from any input to a union relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.operation = CanonicalOperation::Literal
+table LiteralOptions {
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// 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;
+}
+
+/// Frequently there is an open direct channel between producers
+/// of queries (client) and their consumers (server).
+/// `Interactive` is provided to reify this case:
+rpc_service Interactive {

Review comment:
       I believe anything related to gRPC is out of scope for the specification

##########
File path: cpp/build-support/update-flatbuffers.sh
##########
@@ -23,16 +23,18 @@
 CWD="$(cd "$(dirname "${BASH_SOURCE[0]:-$0}")" && pwd)"
 SOURCE_DIR=$CWD/../src
 FORMAT_DIR=$CWD/../../format
-FLATC="flatc -c --cpp-std c++11"
+FLATC="flatc --cpp --grpc --cpp-std c++11"

Review comment:
       I do not think we should generate gRPC extensions here

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);

Review comment:
       It seems simpler to put the field/output type in `Expression` instead of replicating it across each type in `ExpressionImpl`

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction
+}
+
+table Call {
+  /// The function whose invocation this Call represents.
+  function: Function (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `function_name`.
+  arguments: [Expression] (required);
+
+  /// The type of data which invoking `function_name` will return.
+  /// Field is used instead of Type to pick up child fields,
+  /// dictionary encoding, etc.
+  field: Field;
+}
+
+enum CanonicalOperationId : uint32 {
+  Literal,
+  Filter,
+  Project,
+  Aggregate,
+  Join,
+  OrderBy,
+  Limit,
+  Common,
+  Union,
+  InteractiveOutput,
+}
+
+table CanonicalOperation {
+  id: CanonicalOperationId;
+}
+
+table NonCanonicalOperation {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Operation {
+  CanonicalOperation, NonCanonicalOperation
+}
+
+/// A relation is a set of rows with consistent schema.
+table Relation {
+  /// The operation which this Relation wraps.
+  operation: Operation (required);
+
+  /// Parameters for `operation`; content/format may be unique to each
+  /// value of `operation`.
+  options: InlineBuffer;
+
+  /// The arguments passed to `operation`.
+  arguments: [Relation] (required);
+
+  /// The schema of rows in this Relation
+  schema: Schema;
+}
+
+/// The contents of Relation.options will be FilterOptions
+/// if Relation.operation = CanonicalOperation::Filter
+table FilterOptions {
+  /// The expression which will be evaluated against input rows
+  /// to determine whether they should be excluded from the
+  /// filter relation's output.
+  filter_expression: Expression (required);
+}
+
+/// The contents of Relation.options will be ProjectOptions
+/// if Relation.operation = CanonicalOperation::Project
+table ProjectOptions {
+  /// Expressions which will be evaluated to produce to
+  /// the rows of the project relation's output.
+  expressions: [Expression] (required);
+}
+
+/// The contents of Relation.options will be AggregateOptions
+/// if Relation.operation = CanonicalOperation::Aggregate
+table AggregateOptions {
+  /// 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 : uint32 {
+  Inner,
+  LeftOuter,
+  RightOuter,
+  FullOuter,
+  Cross,
+}
+
+table CanonicalJoinKind {
+  id: CanonicalJoinKindId;
+}
+
+table NonCanonicalJoinKind {
+  name_space: string (required);
+  name: string (required);
+}
+
+union JoinKind {
+  CanonicalJoinKind, NonCanonicalJoinKind
+}
+
+/// The contents of Relation.options will be JoinOptions
+/// if Relation.operation = CanonicalOperation::Join
+table JoinOptions {
+  /// 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);
+}
+
+/// 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
+}
+
+table SortKey {
+  value: Expression (required);
+  ordering: Ordering = ASCENDING_THEN_NULLS;
+}
+
+/// The contents of Relation.options will be OrderByOptions
+/// if Relation.operation = CanonicalOperation::OrderBy
+table OrderByOptions {
+  /// Define sort order for rows of output.
+  /// Keys with higher precedence are ordered ahead of other keys.
+  keys: [SortKey] (required);
+}
+
+/// The contents of Relation.options will be LimitOptions
+/// if Relation.operation = CanonicalOperation::Limit
+table LimitOptions {
+  /// Set the maximum number of rows of output.
+  count: long;
+}
+
+/// The contents of Relation.options will be CommonOptions
+/// if Relation.operation = CanonicalOperation::Common
+table CommonOptions {
+  /// 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 contents of Relation.options will be UnionOptions
+/// if Relation.operation = CanonicalOperation::Union
+table UnionOptions {
+  /// For simplicity, all rows from any input to a union relation
+  /// will always be concatenated into a single output- establishing
+  /// uniqueness of output rows is deferred to other relations.
+}
+
+/// The contents of Relation.options will be LiteralOptions
+/// if Relation.operation = CanonicalOperation::Literal
+table LiteralOptions {
+  /// The columns of this literal relation.
+  columns: [Literal] (required);
+}
+
+/// A specification of a query.
+table Plan {
+  /// One or more output relations.
+  sinks: [Relation] (required);

Review comment:
       What is the interpretation of multiple outputs (versus a single output)? 

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,348 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  // Ideally we'd simply have `union Expression { Literal, FieldRef, Call }`
+  // but not all generators support vectors of unions so we provide minimal
+  // indirection to support them.
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  Array, Scalar
+}
+
+table Scalar {}
+
+table Array {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  ///
+  /// Note that this is orthogonal to type and refers to the number
+  /// of rows spanned by this Literal - a Literal may be Scalar shaped
+  /// with multiple "columns" if the type happens to be Struct.
+  shape: Shape (required);
+
+  /// The type of this literal. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field (required);
+
+  /// Buffers containing N elements of arrow-formatted data, where N
+  /// is Array.length if shape is Array or 1 if shape is Scalar.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [InlineBuffer];
+
+  /// If (and only if) this Literal has dictionary type, this field dictionary
+  /// into which the literal's indices refer.
+  dictionary: Literal;
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of the referenced Field. Field is used instead of Type to pick
+  /// up child fields, dictionary encoding, etc.
+  field: Field;
+}
+
+/// A canonical (probably SQL equivalent) function
+enum CanonicalFunctionId : uint32 {
+  // logical
+  And,
+  Not,
+  Or,
+
+  // arithmetic
+  Add,
+  Subtract,
+  Multiply,
+  Divide,
+  Power,
+  AbsoluteValue,
+  Negate,
+  Sign,
+
+  // comparison
+  Equal,
+  NotEqual,
+  Greater,
+  GreaterOrEqual,
+  Less,
+  LessOrEqual,
+
+  // aggregations
+  All,
+  Any,
+  Count,
+  Mean,
+  Min,
+  Max,
+  Mode,
+  Product,
+  Sum,
+  Tdigest,
+  Quantile,
+  Variance,
+  StandardDeviation,
+}
+
+table CanonicalFunction {
+  id: CanonicalFunctionId;
+}
+
+table NonCanonicalFunction {
+  name_space: string (required);
+  name: string (required);
+}
+
+union Function {
+  CanonicalFunction, NonCanonicalFunction

Review comment:
       Presumably we would want to add a UserDefinedFunction here which is able to pass an inline buffer containing the serialized function 

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);
+}
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);

Review comment:
       Per below, if you lift the output type (field) into Expression (if we agree that every Expression needs an output Field — name and type), than you can simply call this `ExpressionOp`, since then `Expression` serves more purpose than simply being wrapper for `Expression.impl`

##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,267 @@
+// 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.flatbuf.computeir;
+
+/// Avoid use of org.apache.arrow.Buffer because it requires a
+/// sidecar block of bytes.
+table InlineBuffer {
+  // ulong is used to guarantee alignment and padding of `bytes` so that flatbuffers
+  // and other alignment sensitive blobs can be stored here
+  bytes: [ulong] (required);

Review comment:
       I notice the `ForceVectorAlignment` function in C++
   
   https://github.com/google/flatbuffers/blob/master/include/flatbuffers/flatbuffers.h#L1755
   
   We should look at whether
   
   ```
   table InlineBuffer {
     padding_for_alignment:ulong = 0 (required);
     bytes:[ubyte] (required);
   }
   ```
   
   guarantees `bytes` to be aligned




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



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

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r698818377



##########
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:
       Why not follow the IEEE754 recommendation?
   
   * Iceberg uses it: https://github.com/apache/iceberg/blob/master/site/docs/spec.md#sorting
   * Java uses it
   * Rust lang follows it in practice (see e.g. https://github.com/rust-lang/rust/issues/5585 and [ord-float](https://crates.io/crates/float-ord))

##########
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 think that the rational is inverted: usually planners (IR producers) are the ones that need to know which signatures the consumers accept, so that they can plan eventual detours (e.g. perform some casts to match signatures).
   
   For example, if a consumer does not support `(add,Date32,Duration(seconds))`, there are ways around this via casting and arithmetics; the planner may want to get around this limitation.
   
   Some examples where typing is needed:
   
   ```
   sum(f32) -> (f32 or f64)?
   sum(i32) -> (i32 or i64)?
   add(i32,i32) -> (i32 or i64)?
   ```
   
   different code will need to run depending on the requested output type. If an IR consumer does not support some of the calls, the producer needs to know that so that it can emit a different set of operators (e.g. perform some casts before and after the main operator).
   
   `DataFusion` dynamic dispatch is based on this idea; the physical planner plans out which physical expressions will be used based on the input schema. Broadly, it goes like this:
   
   1. every operator has an ordered set of valid input types that it supports (e.g. `add` could be `(i32,i32),(i64,i64),(f32,f32),(f64,f64)` in this order; ordering is from least informative (in Shannon's notion) to most informative.
   2. for every operator, there is a map between input and output: `output_type(op,arg_types) -> arg_type` (e.g. `output_type("add", [i32,i32]) -> i64`)
   3. the first lossless coercion to fit the signature is selected (e.g. if input types are `(i32,i64)`, they are mapped to `(i64,i64)` prior to being passed to `add` (since the `(i32,i32)` would be lossy)).
   
   Given an input schema (vector of of `DataType`), all IR calls are known in advance via the type resolution of the physical nodes available + applied coercion.  `DataFusion`'s planner knows in advance which physical nodes are implemented and uses this information to plan out the operations (since it is in the same source code). UDFs and UDAFs in DataFusion must register this information; in fact, UDFs and UDAFs use the same signatures as builtin functions; the only way they are special is that their implementation is not known to DataFusion and thus can't be inlined (which is fine).
   
   I would imagine that consumers would have a registry of implemented functions, which they would inform producers about _before planning_, so that the producer can error out or "dodge via cast" _during planning_. The producer would use the registry to decide which physical plan it would pass down to the consumer, pretty much like an architecture that declares the registry in their user manual and that compilers use.
   
   The main question for me is whether we want to use logical types or physical types. E.g. is it relevant that a consumer does not implement `add(date32,duration(seconds))` if it implements `add(int32,int64)`? My understanding is that it should not matter since logical semantics can be handled by an IR planner.




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



[GitHub] [arrow] github-actions[bot] commented on pull request #10934: ARROW-14062: [Format] Initial arrow-internal specification of compute IR

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#issuecomment-924368794






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



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

Posted by GitBox <gi...@apache.org>.
cpcloud commented on a change in pull request #10934:
URL: https://github.com/apache/arrow/pull/10934#discussion_r689825291



##########
File path: format/ComputeIR.fbs
##########
@@ -0,0 +1,241 @@
+// 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 "Message.fbs";
+
+namespace org.apache.arrow.flatbuf.computeir;
+
+/// An expression is one of
+/// - a Literal datum
+/// - a reference to a Field from a Relation
+/// - a call to a named function
+/// On evaluation, an Expression will have either array or scalar shape.
+union ExpressionImpl {
+  Literal, FieldRef, Call
+}
+
+table Expression {
+  impl: ExpressionImpl (required);
+}
+
+union Shape {
+  ArrayShape, ScalarShape
+}
+
+table ScalarShape {}
+
+table ArrayShape {
+  /// Number of slots.
+  length: long;
+}
+
+table Literal {
+  /// Shape of this literal.
+  shape: Shape (required);
+
+  /// The type of this literal.
+  type: Type (required);
+
+  /// Buffers containing `length` elements of arrow-formatted data.
+  /// If `length` is absent (this Literal is scalar), these buffers
+  /// are sized to accommodate a single element of arrow-formatted data.
+  /// XXX this can be optimized for trivial scalars later
+  buffers: [Buffer];
+}
+
+table FieldRef {
+  /// A sequence of field names to allow referencing potentially nested fields
+  path: [string];
+
+  /// 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;
+
+  /// The type of data in the referenced Field.
+  type: Type;
+}
+
+table Call {
+  /// The name of the function whose invocation this Call represents.
+  function_name: string (required);
+
+  /// Parameters for `function_name`; content/format may be unique to each
+  /// value of `function_name`.
+  options: Buffer;

Review comment:
       Thanks for the clarification.




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