You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2020/02/13 02:12:10 UTC

[calcite] 01/03: Add RelBuilder.transform, which allows you to clone a RelBuilder with slightly different Config

This is an automated email from the ASF dual-hosted git repository.

jhyde pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit 051b6919dfc5b60406e81d5e8b8d5efb263def87
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Fri Feb 7 18:10:42 2020 -0800

    Add RelBuilder.transform, which allows you to clone a RelBuilder with slightly different Config
    
    Add class RelFactories.Struct, which contains an instance of
    each RelNode factory. This allows more efficient initialization
    of RelBuilder.
---
 .../org/apache/calcite/rel/core/RelFactories.java  | 132 +++++++++++++--
 .../java/org/apache/calcite/tools/RelBuilder.java  | 184 +++++++++------------
 2 files changed, 193 insertions(+), 123 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index 13b2cfd..e077008 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.rel.core;
 
 import org.apache.calcite.linq4j.function.Experimental;
+import org.apache.calcite.plan.Context;
 import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
@@ -53,6 +54,7 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
@@ -60,6 +62,7 @@ import com.google.common.collect.ImmutableSet;
 import java.lang.reflect.Type;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.SortedSet;
 import javax.annotation.Nonnull;
@@ -116,24 +119,28 @@ public class RelFactories {
   public static final RepeatUnionFactory DEFAULT_REPEAT_UNION_FACTORY =
       new RepeatUnionFactoryImpl();
 
+  public static final Struct DEFAULT_STRUCT =
+      new Struct(DEFAULT_FILTER_FACTORY,
+          DEFAULT_PROJECT_FACTORY,
+          DEFAULT_AGGREGATE_FACTORY,
+          DEFAULT_SORT_FACTORY,
+          DEFAULT_EXCHANGE_FACTORY,
+          DEFAULT_SORT_EXCHANGE_FACTORY,
+          DEFAULT_SET_OP_FACTORY,
+          DEFAULT_JOIN_FACTORY,
+          DEFAULT_CORRELATE_FACTORY,
+          DEFAULT_VALUES_FACTORY,
+          DEFAULT_TABLE_SCAN_FACTORY,
+          DEFAULT_TABLE_FUNCTION_SCAN_FACTORY,
+          DEFAULT_SNAPSHOT_FACTORY,
+          DEFAULT_MATCH_FACTORY,
+          DEFAULT_SPOOL_FACTORY,
+          DEFAULT_REPEAT_UNION_FACTORY);
+
   /** A {@link RelBuilderFactory} that creates a {@link RelBuilder} that will
    * create logical relational expressions for everything. */
   public static final RelBuilderFactory LOGICAL_BUILDER =
-      RelBuilder.proto(
-          Contexts.of(DEFAULT_PROJECT_FACTORY,
-              DEFAULT_FILTER_FACTORY,
-              DEFAULT_JOIN_FACTORY,
-              DEFAULT_SORT_FACTORY,
-              DEFAULT_EXCHANGE_FACTORY,
-              DEFAULT_SORT_EXCHANGE_FACTORY,
-              DEFAULT_AGGREGATE_FACTORY,
-              DEFAULT_MATCH_FACTORY,
-              DEFAULT_SET_OP_FACTORY,
-              DEFAULT_VALUES_FACTORY,
-              DEFAULT_TABLE_SCAN_FACTORY,
-              DEFAULT_SNAPSHOT_FACTORY,
-              DEFAULT_SPOOL_FACTORY,
-              DEFAULT_REPEAT_UNION_FACTORY));
+      RelBuilder.proto(Contexts.of(DEFAULT_STRUCT));
 
   private RelFactories() {
   }
@@ -673,4 +680,99 @@ public class RelFactories {
       return LogicalRepeatUnion.create(seed, iterative, all, iterationLimit);
     }
   }
+
+  /** Immutable record that contains an instance of each factory. */
+  public static class Struct {
+    public final FilterFactory filterFactory;
+    public final ProjectFactory projectFactory;
+    public final AggregateFactory aggregateFactory;
+    public final SortFactory sortFactory;
+    public final ExchangeFactory exchangeFactory;
+    public final SortExchangeFactory sortExchangeFactory;
+    public final SetOpFactory setOpFactory;
+    public final JoinFactory joinFactory;
+    public final CorrelateFactory correlateFactory;
+    public final ValuesFactory valuesFactory;
+    public final TableScanFactory scanFactory;
+    public final TableFunctionScanFactory tableFunctionScanFactory;
+    public final SnapshotFactory snapshotFactory;
+    public final MatchFactory matchFactory;
+    public final SpoolFactory spoolFactory;
+    public final RepeatUnionFactory repeatUnionFactory;
+
+    private Struct(FilterFactory filterFactory,
+        ProjectFactory projectFactory,
+        AggregateFactory aggregateFactory,
+        SortFactory sortFactory,
+        ExchangeFactory exchangeFactory,
+        SortExchangeFactory sortExchangeFactory,
+        SetOpFactory setOpFactory,
+        JoinFactory joinFactory,
+        CorrelateFactory correlateFactory,
+        ValuesFactory valuesFactory,
+        TableScanFactory scanFactory,
+        TableFunctionScanFactory tableFunctionScanFactory,
+        SnapshotFactory snapshotFactory,
+        MatchFactory matchFactory,
+        SpoolFactory spoolFactory,
+        RepeatUnionFactory repeatUnionFactory) {
+      this.filterFactory = Objects.requireNonNull(filterFactory);
+      this.projectFactory = Objects.requireNonNull(projectFactory);
+      this.aggregateFactory = Objects.requireNonNull(aggregateFactory);
+      this.sortFactory = Objects.requireNonNull(sortFactory);
+      this.exchangeFactory = Objects.requireNonNull(exchangeFactory);
+      this.sortExchangeFactory = Objects.requireNonNull(sortExchangeFactory);
+      this.setOpFactory = Objects.requireNonNull(setOpFactory);
+      this.joinFactory = Objects.requireNonNull(joinFactory);
+      this.correlateFactory = Objects.requireNonNull(correlateFactory);
+      this.valuesFactory = Objects.requireNonNull(valuesFactory);
+      this.scanFactory = Objects.requireNonNull(scanFactory);
+      this.tableFunctionScanFactory =
+          Objects.requireNonNull(tableFunctionScanFactory);
+      this.snapshotFactory = Objects.requireNonNull(snapshotFactory);
+      this.matchFactory = Objects.requireNonNull(matchFactory);
+      this.spoolFactory = Objects.requireNonNull(spoolFactory);
+      this.repeatUnionFactory = Objects.requireNonNull(repeatUnionFactory);
+    }
+
+    public static @Nonnull Struct fromContext(Context context) {
+      Struct struct = context.unwrap(Struct.class);
+      if (struct != null) {
+        return struct;
+      }
+      return new Struct(
+          Util.first(context.unwrap(FilterFactory.class),
+              DEFAULT_FILTER_FACTORY),
+          Util.first(context.unwrap(ProjectFactory.class),
+              DEFAULT_PROJECT_FACTORY),
+          Util.first(context.unwrap(AggregateFactory.class),
+              DEFAULT_AGGREGATE_FACTORY),
+          Util.first(context.unwrap(SortFactory.class),
+              DEFAULT_SORT_FACTORY),
+          Util.first(context.unwrap(ExchangeFactory.class),
+              DEFAULT_EXCHANGE_FACTORY),
+          Util.first(context.unwrap(SortExchangeFactory.class),
+              DEFAULT_SORT_EXCHANGE_FACTORY),
+          Util.first(context.unwrap(SetOpFactory.class),
+              DEFAULT_SET_OP_FACTORY),
+          Util.first(context.unwrap(JoinFactory.class),
+              DEFAULT_JOIN_FACTORY),
+          Util.first(context.unwrap(CorrelateFactory.class),
+              DEFAULT_CORRELATE_FACTORY),
+          Util.first(context.unwrap(ValuesFactory.class),
+              DEFAULT_VALUES_FACTORY),
+          Util.first(context.unwrap(TableScanFactory.class),
+              DEFAULT_TABLE_SCAN_FACTORY),
+          Util.first(context.unwrap(TableFunctionScanFactory.class),
+              DEFAULT_TABLE_FUNCTION_SCAN_FACTORY),
+          Util.first(context.unwrap(SnapshotFactory.class),
+              DEFAULT_SNAPSHOT_FACTORY),
+          Util.first(context.unwrap(MatchFactory.class),
+              DEFAULT_MATCH_FACTORY),
+          Util.first(context.unwrap(SpoolFactory.class),
+              DEFAULT_SPOOL_FACTORY),
+          Util.first(context.unwrap(RepeatUnionFactory.class),
+              DEFAULT_REPEAT_UNION_FACTORY));
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index b4455a0..e4ef056 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -119,6 +119,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.function.UnaryOperator;
 import java.util.stream.Collectors;
 import javax.annotation.Nonnull;
 
@@ -143,25 +144,10 @@ import static org.apache.calcite.util.Static.RESOURCE;
 public class RelBuilder {
   protected final RelOptCluster cluster;
   protected final RelOptSchema relOptSchema;
-  private final RelFactories.FilterFactory filterFactory;
-  private final RelFactories.ProjectFactory projectFactory;
-  private final RelFactories.AggregateFactory aggregateFactory;
-  private final RelFactories.SortFactory sortFactory;
-  private final RelFactories.ExchangeFactory exchangeFactory;
-  private final RelFactories.SortExchangeFactory sortExchangeFactory;
-  private final RelFactories.SetOpFactory setOpFactory;
-  private final RelFactories.JoinFactory joinFactory;
-  private final RelFactories.CorrelateFactory correlateFactory;
-  private final RelFactories.ValuesFactory valuesFactory;
-  private final RelFactories.TableScanFactory scanFactory;
-  private final RelFactories.TableFunctionScanFactory tableFunctionScanFactory;
-  private final RelFactories.SnapshotFactory snapshotFactory;
-  private final RelFactories.MatchFactory matchFactory;
-  private final RelFactories.SpoolFactory spoolFactory;
-  private final RelFactories.RepeatUnionFactory repeatUnionFactory;
   private final Deque<Frame> stack = new ArrayDeque<>();
   private final RexSimplify simplifier;
   private final Config config;
+  private final RelFactories.Struct struct;
 
   protected RelBuilder(Context context, RelOptCluster cluster,
       RelOptSchema relOptSchema) {
@@ -171,54 +157,8 @@ public class RelBuilder {
       context = Contexts.EMPTY_CONTEXT;
     }
     this.config = getConfig(context);
-    this.aggregateFactory =
-        Util.first(context.unwrap(RelFactories.AggregateFactory.class),
-            RelFactories.DEFAULT_AGGREGATE_FACTORY);
-    this.filterFactory =
-        Util.first(context.unwrap(RelFactories.FilterFactory.class),
-            RelFactories.DEFAULT_FILTER_FACTORY);
-    this.projectFactory =
-        Util.first(context.unwrap(RelFactories.ProjectFactory.class),
-            RelFactories.DEFAULT_PROJECT_FACTORY);
-    this.sortFactory =
-        Util.first(context.unwrap(RelFactories.SortFactory.class),
-            RelFactories.DEFAULT_SORT_FACTORY);
-    this.exchangeFactory =
-        Util.first(context.unwrap(RelFactories.ExchangeFactory.class),
-            RelFactories.DEFAULT_EXCHANGE_FACTORY);
-    this.sortExchangeFactory =
-        Util.first(context.unwrap(RelFactories.SortExchangeFactory.class),
-            RelFactories.DEFAULT_SORT_EXCHANGE_FACTORY);
-    this.setOpFactory =
-        Util.first(context.unwrap(RelFactories.SetOpFactory.class),
-            RelFactories.DEFAULT_SET_OP_FACTORY);
-    this.joinFactory =
-        Util.first(context.unwrap(RelFactories.JoinFactory.class),
-            RelFactories.DEFAULT_JOIN_FACTORY);
-    this.correlateFactory =
-        Util.first(context.unwrap(RelFactories.CorrelateFactory.class),
-            RelFactories.DEFAULT_CORRELATE_FACTORY);
-    this.valuesFactory =
-        Util.first(context.unwrap(RelFactories.ValuesFactory.class),
-            RelFactories.DEFAULT_VALUES_FACTORY);
-    this.scanFactory =
-        Util.first(context.unwrap(RelFactories.TableScanFactory.class),
-            RelFactories.DEFAULT_TABLE_SCAN_FACTORY);
-    this.tableFunctionScanFactory =
-        Util.first(context.unwrap(RelFactories.TableFunctionScanFactory.class),
-            RelFactories.DEFAULT_TABLE_FUNCTION_SCAN_FACTORY);
-    this.snapshotFactory =
-        Util.first(context.unwrap(RelFactories.SnapshotFactory.class),
-            RelFactories.DEFAULT_SNAPSHOT_FACTORY);
-    this.matchFactory =
-        Util.first(context.unwrap(RelFactories.MatchFactory.class),
-            RelFactories.DEFAULT_MATCH_FACTORY);
-    this.spoolFactory =
-        Util.first(context.unwrap(RelFactories.SpoolFactory.class),
-            RelFactories.DEFAULT_SPOOL_FACTORY);
-    this.repeatUnionFactory =
-        Util.first(context.unwrap(RelFactories.RepeatUnionFactory.class),
-            RelFactories.DEFAULT_REPEAT_UNION_FACTORY);
+    this.struct =
+        Objects.requireNonNull(RelFactories.Struct.fromContext(context));
     final RexExecutor executor =
         Util.first(context.unwrap(RexExecutor.class),
             Util.first(cluster.getPlanner().getExecutor(), RexUtil.EXECUTOR));
@@ -246,6 +186,14 @@ public class RelBuilder {
             new RelBuilder(config.getContext(), cluster, relOptSchema));
   }
 
+  /** Creates a copy of this RelBuilder, with the same state as this, applying
+   * a transform to the config. */
+  public RelBuilder transform(UnaryOperator<Config> transform) {
+    final Context context =
+        Contexts.of(struct, transform.apply(config));
+    return new RelBuilder(context, cluster, relOptSchema);
+  }
+
   /** Converts this RelBuilder to a string.
    * The string is the string representation of all of the RelNodes on the stack. */
   @Override public String toString() {
@@ -284,7 +232,7 @@ public class RelBuilder {
   }
 
   public RelFactories.TableScanFactory getScanFactory() {
-    return scanFactory;
+    return struct.scanFactory;
   }
 
   // Methods for manipulating the stack
@@ -1071,7 +1019,9 @@ public class RelBuilder {
     if (relOptTable == null) {
       throw RESOURCE.tableNotFound(String.join(".", names)).ex();
     }
-    final RelNode scan = scanFactory.createScan(cluster, relOptTable, ImmutableList.of());
+    final RelNode scan =
+        struct.scanFactory.createScan(cluster, relOptTable,
+            ImmutableList.of());
     push(scan);
     rename(relOptTable.getRowType().getFieldNames());
 
@@ -1104,7 +1054,8 @@ public class RelBuilder {
    */
   public RelBuilder snapshot(RexNode period) {
     final Frame frame = stack.pop();
-    final RelNode snapshot = snapshotFactory.createSnapshot(frame.rel, period);
+    final RelNode snapshot =
+        struct.snapshotFactory.createSnapshot(frame.rel, period);
     stack.push(new Frame(snapshot, frame.fields));
     return this;
   }
@@ -1164,8 +1115,8 @@ public class RelBuilder {
 
     final RexNode call = call(operator, ImmutableList.copyOf(operands));
     final RelNode functionScan =
-        tableFunctionScanFactory.createTableFunctionScan(cluster, inputs,
-            call, null, getColumnMappings(operator));
+        struct.tableFunctionScanFactory.createTableFunctionScan(cluster,
+            inputs, call, null, getColumnMappings(operator));
     push(functionScan);
     return this;
   }
@@ -1218,8 +1169,9 @@ public class RelBuilder {
 
     if (!simplifiedPredicates.isAlwaysTrue()) {
       final Frame frame = stack.pop();
-      final RelNode filter = filterFactory.createFilter(frame.rel,
-          simplifiedPredicates, ImmutableSet.copyOf(variablesSet));
+      final RelNode filter =
+          struct.filterFactory.createFilter(frame.rel,
+              simplifiedPredicates, ImmutableSet.copyOf(variablesSet));
       stack.push(new Frame(filter, frame.fields));
     }
     return this;
@@ -1478,7 +1430,7 @@ public class RelBuilder {
       return this;
     }
     final RelNode project =
-        projectFactory.createProject(frame.rel,
+        struct.projectFactory.createProject(frame.rel,
             ImmutableList.copyOf(hints),
             ImmutableList.copyOf(nodeList),
             fieldNameList);
@@ -1771,8 +1723,9 @@ public class RelBuilder {
       ImmutableList<ImmutableBitSet> groupSets, RelNode input,
       List<AggregateCall> aggregateCalls, List<RexNode> extraNodes,
       ImmutableList<Field> inFields) {
-    final RelNode aggregate = aggregateFactory.createAggregate(input,
-        ImmutableList.of(), groupSet, groupSets, aggregateCalls);
+    final RelNode aggregate =
+        struct.aggregateFactory.createAggregate(input,
+            ImmutableList.of(), groupSet, groupSets, aggregateCalls);
 
     // build field list
     final ImmutableList.Builder<Field> fields = ImmutableList.builder();
@@ -1829,7 +1782,7 @@ public class RelBuilder {
     case 1:
       return push(inputs.get(0));
     default:
-      return push(setOpFactory.createSetOp(kind, inputs, all));
+      return push(struct.setOpFactory.createSetOp(kind, inputs, all));
     }
   }
 
@@ -1914,7 +1867,9 @@ public class RelBuilder {
         rowType,
         transientTable,
         ImmutableList.of(tableName));
-    RelNode scan = scanFactory.createScan(cluster, relOptTable, ImmutableList.of());
+    RelNode scan =
+        struct.scanFactory.createScan(cluster, relOptTable,
+            ImmutableList.of());
     push(scan);
     rename(rowType.getFieldNames());
     return this;
@@ -1927,8 +1882,11 @@ public class RelBuilder {
    * @param writeType Spool's write type (as described in {@link Spool.Type})
    * @param table Table to write into
    */
-  private RelBuilder tableSpool(Spool.Type readType, Spool.Type writeType, RelOptTable table) {
-    RelNode spool =  spoolFactory.createTableSpool(peek(), readType, writeType, table);
+  private RelBuilder tableSpool(Spool.Type readType, Spool.Type writeType,
+      RelOptTable table) {
+    RelNode spool =
+        struct.spoolFactory.createTableSpool(peek(), readType, writeType,
+            table);
     replaceTop(spool);
     return this;
   }
@@ -1984,8 +1942,10 @@ public class RelBuilder {
 
     RelNode iterative = tableSpool(Spool.Type.LAZY, Spool.Type.LAZY, finder.relOptTable).build();
     RelNode seed = tableSpool(Spool.Type.LAZY, Spool.Type.LAZY, finder.relOptTable).build();
-    RelNode repUnion = repeatUnionFactory.createRepeatUnion(seed, iterative, all, iterationLimit);
-    return push(repUnion);
+    RelNode repeatUnion =
+        struct.repeatUnionFactory.createRepeatUnion(seed, iterative, all,
+            iterationLimit);
+    return push(repeatUnion);
   }
 
   /**
@@ -2065,11 +2025,13 @@ public class RelBuilder {
       default:
         postCondition = condition;
       }
-      join = correlateFactory.createCorrelate(left.rel, right.rel, id,
-          requiredColumns, joinType);
+      join =
+          struct.correlateFactory.createCorrelate(left.rel, right.rel, id,
+              requiredColumns, joinType);
     } else {
-      join = joinFactory.createJoin(left.rel, right.rel, ImmutableList.of(), condition,
-          variablesSet, joinType, false);
+      join =
+          struct.joinFactory.createJoin(left.rel, right.rel,
+              ImmutableList.of(), condition, variablesSet, joinType, false);
     }
     final ImmutableList.Builder<Field> fields = ImmutableList.builder();
     fields.addAll(left.fields);
@@ -2102,9 +2064,9 @@ public class RelBuilder {
     rename(registrar.names);
     Frame left = stack.pop();
 
-    final RelNode correlate = correlateFactory
-        .createCorrelate(left.rel, right.rel, correlationId,
-            ImmutableBitSet.of(requiredOrdinals), joinType);
+    final RelNode correlate =
+        struct.correlateFactory.createCorrelate(left.rel, right.rel,
+            correlationId, ImmutableBitSet.of(requiredOrdinals), joinType);
 
     final ImmutableList.Builder<Field> fields = ImmutableList.builder();
     fields.addAll(left.fields);
@@ -2154,7 +2116,7 @@ public class RelBuilder {
   public RelBuilder semiJoin(Iterable<? extends RexNode> conditions) {
     final Frame right = stack.pop();
     final RelNode semiJoin =
-        joinFactory.createJoin(peek(),
+        struct.joinFactory.createJoin(peek(),
             right.rel,
             ImmutableList.of(),
             and(conditions),
@@ -2191,7 +2153,7 @@ public class RelBuilder {
   public RelBuilder antiJoin(Iterable<? extends RexNode> conditions) {
     final Frame right = stack.pop();
     final RelNode antiJoin =
-        joinFactory.createJoin(peek(),
+        struct.joinFactory.createJoin(peek(),
             right.rel,
             ImmutableList.of(),
             and(conditions),
@@ -2311,7 +2273,8 @@ public class RelBuilder {
   public RelBuilder empty() {
     final Frame frame = stack.pop();
     final RelNode values =
-        valuesFactory.createValues(cluster, frame.rel.getRowType(), ImmutableList.of());
+        struct.valuesFactory.createValues(cluster, frame.rel.getRowType(),
+            ImmutableList.of());
     stack.push(new Frame(values, frame.fields));
     return this;
   }
@@ -2328,8 +2291,9 @@ public class RelBuilder {
   public RelBuilder values(RelDataType rowType, Object... columnValues) {
     final ImmutableList<ImmutableList<RexLiteral>> tupleList =
         tupleList(rowType.getFieldCount(), columnValues);
-    RelNode values = valuesFactory.createValues(cluster, rowType,
-        ImmutableList.copyOf(tupleList));
+    RelNode values =
+        struct.valuesFactory.createValues(cluster, rowType,
+            ImmutableList.copyOf(tupleList));
     push(values);
     return this;
   }
@@ -2346,7 +2310,8 @@ public class RelBuilder {
   public RelBuilder values(Iterable<? extends List<RexLiteral>> tupleList,
       RelDataType rowType) {
     RelNode values =
-        valuesFactory.createValues(cluster, rowType, copy(tupleList));
+        struct.valuesFactory.createValues(cluster, rowType,
+            copy(tupleList));
     push(values);
     return this;
   }
@@ -2390,7 +2355,8 @@ public class RelBuilder {
 
   /** Creates an Exchange by distribution. */
   public RelBuilder exchange(RelDistribution distribution) {
-    RelNode exchange = exchangeFactory.createExchange(peek(), distribution);
+    RelNode exchange =
+        struct.exchangeFactory.createExchange(peek(), distribution);
     replaceTop(exchange);
     return this;
   }
@@ -2398,8 +2364,9 @@ public class RelBuilder {
   /** Creates a SortExchange by distribution and collation. */
   public RelBuilder sortExchange(RelDistribution distribution,
       RelCollation collation) {
-    RelNode exchange = sortExchangeFactory
-        .createSortExchange(peek(), distribution, collation);
+    RelNode exchange =
+        struct.sortExchangeFactory.createSortExchange(peek(), distribution,
+            collation);
     replaceTop(exchange);
     return this;
   }
@@ -2461,7 +2428,7 @@ public class RelBuilder {
         if (sort2.offset == null && sort2.fetch == null) {
           replaceTop(sort2.getInput());
           final RelNode sort =
-              sortFactory.createSort(peek(), sort2.collation,
+              struct.sortFactory.createSort(peek(), sort2.collation,
                   offsetNode, fetchNode);
           replaceTop(sort);
           return this;
@@ -2473,10 +2440,10 @@ public class RelBuilder {
           final Sort sort2 = (Sort) project.getInput();
           if (sort2.offset == null && sort2.fetch == null) {
             final RelNode sort =
-                sortFactory.createSort(sort2.getInput(), sort2.collation,
-                    offsetNode, fetchNode);
+                struct.sortFactory.createSort(sort2.getInput(),
+                    sort2.collation, offsetNode, fetchNode);
             replaceTop(
-                projectFactory.createProject(sort,
+                struct.projectFactory.createProject(sort,
                     project.getHints(),
                     project.getProjects(),
                     Pair.right(project.getNamedProjects())));
@@ -2489,8 +2456,8 @@ public class RelBuilder {
       project(registrar.extraNodes);
     }
     final RelNode sort =
-        sortFactory.createSort(peek(), RelCollations.of(fieldCollations),
-            offsetNode, fetchNode);
+        struct.sortFactory.createSort(peek(),
+            RelCollations.of(fieldCollations), offsetNode, fetchNode);
     replaceTop(sort);
     if (registrar.addedFieldCount() > 0) {
       project(registrar.originalExtraNodes);
@@ -2534,7 +2501,8 @@ public class RelBuilder {
   public RelBuilder convert(RelDataType castRowType, boolean rename) {
     final RelNode r = build();
     final RelNode r2 =
-        RelOptUtil.createCastRel(r, castRowType, rename, projectFactory);
+        RelOptUtil.createCastRel(r, castRowType, rename,
+            struct.projectFactory);
     push(r2);
     return this;
   }
@@ -2594,11 +2562,11 @@ public class RelBuilder {
       measures.put(alias, operands.get(0));
     }
 
-    final RelNode match = matchFactory.createMatch(peek(), pattern,
-        typeBuilder.build(), strictStart, strictEnd, patternDefinitions,
-        measures.build(), after, subsets, allRows,
-        partitionBitSet, RelCollations.of(fieldCollations),
-        interval);
+    final RelNode match =
+        struct.matchFactory.createMatch(peek(), pattern,
+            typeBuilder.build(), strictStart, strictEnd, patternDefinitions,
+            measures.build(), after, subsets, allRows,
+            partitionBitSet, RelCollations.of(fieldCollations), interval);
     stack.push(new Frame(match));
     return this;
   }