You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by GitBox <gi...@apache.org> on 2021/01/18 16:32:21 UTC

[GitHub] [calcite] rubenada opened a new pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

rubenada opened a new pull request #2329:
URL: https://github.com/apache/calcite/pull/2329


   Jira: [CALCITE-3221](https://issues.apache.org/jira/browse/CALCITE-3221)
   - New operator EnumerableMergeUnion
   - New rule EnumerableMergeUnionRule
   - Implementation via EnumerableDefaults#mergeUnion
   - Unit tests


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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561131278



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       collation is the first field (deptId), so $0:
   ```
   UNION, collation[0]
     Input1, collation[0] = Emp(10, Bob), Emp(20, Joe), Emp(20, Greg)
     Input2, collation[0] = Emp(15, Laura), Emp(20, Joe)
   ```




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561945980



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       I fear that you suggestion might trigger a NPE inside `sortKeySelector.apply` if we pass a null element e1/e2.
   BTW `sortComparator.compare` should be able to handle (in some cases) null keys (e.g. when we have nulls first or nulls last, and the comparator must handle nulls accordingly when compared to a non-null value).




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562006125



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,
+        Comparator<@Nullable TKey> sortComparator,
+        boolean all,
+        EqualityComparer<TSource> equalityComparer) {
+      this.sortKeySelector = sortKeySelector;
+      this.sortComparator = sortComparator;
+
+      if (all) {
+        this.processed = null;
+        this.wrapper = null;
+      } else {
+        this.processed = new HashSet<>();
+        this.wrapper = wrapperFor(equalityComparer);
+      }
+
+      final int size = sources.size();
+      this.inputs = new ArrayList<>(size);
+      for (Enumerable<TSource> source : sources) {
+        this.inputs.add(source.enumerator());
+      }
+
+      this.currentInputsValues = (TSource[]) new Object[size];
+      this.activeInputs = this.currentInputsValues.length;
+      this.currentValue = (TSource) NOT_INIT;
+
+      this.initEnumerators();
+    }
+
+    private void initEnumerators() {
+      for (int i = 0; i < this.inputs.size(); i++) {
+        this.moveEnumerator(i);
+      }
+    }
+
+    private void moveEnumerator(int i) {
+      final Enumerator<TSource> enumerator = this.inputs.get(i);
+      if (!enumerator.moveNext()) {
+        this.activeInputs--;
+        this.currentInputsValues[i] = (TSource) FINISHED;
+      } else {
+        this.currentInputsValues[i] = enumerator.current();
+      }
+    }
+
+    @SuppressWarnings("dereference.of.nullable")

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.

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572041385



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       Doing so raises a problem in CheckerFramework:
   ```
   ...\calcite\linq4j\src\main\java\org\apache\calcite\linq4j\MergeUnionEnumerator.java:102: error: [assignment.type.incompatible] incompatible types in assignment.
         currentInputsValues[i] = null;
                                  ^
     found   : @Initialized @Nullable null
     required: TSource[ extends @Initialized @Nullable Object super @Initialized @NonNull Void]
   ```
   @vlsi is there a way to specify that the elements inside `TSource[] currentInputsValues` can be `null`? Or should we suppress warning in this case?




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561681978



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);
+      inputs.add(
+          convert(newInput, newInput.getTraitSet().replace(EnumerableConvention.INSTANCE)));
+    }
+
+    RelNode result = EnumerableMergeUnion.create(sort.getCollation(), inputs, union.all);
+
+    // If Sort contained a LIMIT / OFFSET, then put it back as an EnumerableLimit.
+    // The output of the MergeUnion is already sorted, so we do not need a sort anymore.
+    if (sort.offset != null || sort.fetch != null) {
+      result = EnumerableLimit.create(result, sort.offset, sort.fetch);

Review comment:
       This is already taken care of by E`numerableLimit#create` method, which calls `RelMdCollation.limit(mq, input)` (which just takes the collation from the limit's input).




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560925253



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       Did you really mean `TKey must always be nullable`?
   
   Just in case:
   
   `public static <@Nullable TKey>` means `all instantiations must use nullable type for the key`.
   
   `public static <TKey extends @Nullable Object>` means `users could use either nullable or non-nullable implementations`
   
   `public static <TKey>` is the same as `public static <TKey extends @Nullable Object>`.
   
   `public static <TKey extends Object>`  means `TKey must be non-nullable`.
   




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572041385



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       Doing so raises a problem in CheckerFramework:
   ```
   C:\work\calcite_gradle\calcite\linq4j\src\main\java\org\apache\calcite\linq4j\MergeUnionEnumerator.java:102: error: [assignment.type.incompatible] incompatible types in assignment.
         currentInputsValues[i] = null;
                                  ^
     found   : @Initialized @Nullable null
     required: TSource[ extends @Initialized @Nullable Object super @Initialized @NonNull Void]
   ```
   @vlsi is there a way to specify that the elements inside `TSource[] currentInputsValues` can be `null`? Or should we suppress warning in this case?




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560929944



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,

Review comment:
       Why do you force clients to use `nullable` type for the function?
   I guess `Function1<TSource, TKey>` (or `Function1<? super TSource, ? extends TKey>`) could be enough here.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561146943



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560926612



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")

Review comment:
       Could you suppress individual statements rather than all the warnings in the class?




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561116349



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       Actually, this breaks some tests, I am afraid the set is needed, e.g. a UNION sorted by deptId:
   ```
   UNION
     Input1 = Emp(10, Bob), Emp(20, Joe), Emp(20, Greg)
     Input2 = Emp(15, Laura), Emp(20, Joe)
   ```
    
   Should return: `Emp(10, Bob), Emp(15, Laura), Emp(20, Joe), Emp(20, Greg)`
   At the time the "second" `Emp(20, Joe)` (from input2) is being processed, the previous value is `Emp(20, Greg)`, so if we just save the previous (and not a full set with all previous values), `Emp(20, Joe)` would be incorrectly returned twice.
   




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562008501



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       > and (it's not so easy to guess) I believe a null in there can lead to a NPE.
   
   `linq4j` is a general-purpose library, and it must not depend on `PhysType` quirks




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561920915



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,
+        Comparator<@Nullable TKey> sortComparator,
+        boolean all,
+        EqualityComparer<TSource> equalityComparer) {
+      this.sortKeySelector = sortKeySelector;
+      this.sortComparator = sortComparator;
+
+      if (all) {
+        this.processed = null;
+        this.wrapper = null;
+      } else {
+        this.processed = new HashSet<>();
+        this.wrapper = wrapperFor(equalityComparer);
+      }
+
+      final int size = sources.size();
+      this.inputs = new ArrayList<>(size);
+      for (Enumerable<TSource> source : sources) {
+        this.inputs.add(source.enumerator());
+      }
+
+      this.currentInputsValues = (TSource[]) new Object[size];
+      this.activeInputs = this.currentInputsValues.length;
+      this.currentValue = (TSource) NOT_INIT;
+
+      this.initEnumerators();
+    }
+
+    private void initEnumerators() {
+      for (int i = 0; i < this.inputs.size(); i++) {
+        this.moveEnumerator(i);
+      }
+    }
+
+    private void moveEnumerator(int i) {
+      final Enumerator<TSource> enumerator = this.inputs.get(i);
+      if (!enumerator.moveNext()) {
+        this.activeInputs--;
+        this.currentInputsValues[i] = (TSource) FINISHED;
+      } else {
+        this.currentInputsValues[i] = enumerator.current();
+      }
+    }
+
+    @SuppressWarnings("dereference.of.nullable")

Review comment:
       If you intend to suppress a single case only, please use `SuppressWarnings` for a local variable declaration like in
   
   ```java
   @SuppressWarnings("dereference.of.nullable")
   final Wrapped<TSource> wrapped = this.wrapper.apply(value);
   ```
   
   

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

Review comment:
       In general,`suppressions` should be targeted. If you add `SuppressWarnings` to the full constructor, it might allow unexpected warnings to creep in.
   
   In other words, suppressions should be confined to their minimal scope.
   
   For instance,
   
   ```java
       @RequiresNonNull("inputs")
       @SuppressWarnings("method.invocation.invalid")
       private void initEnumerators(
           EnumerableDefaults.@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this
       ) {
         for (int i = 0; i < this.inputs.size(); i++) {
           this.moveEnumerator(i);
         }
       }
   ```

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       ```java
       private int compare(TSource e1, TSource e2) {
         final TKey key1 = e1 == null ? null : this.sortKeySelector.apply(e1);
         final TKey key2 = e2 == null ? null : this.sortKeySelector.apply(e2);
         return this.sortComparator.compare(key1, key2);
       }
   ```
   
   Ok, what does the error mean here?
   It says that the comparison key can become `null` even in the case `sortKeySelector` never returns nulls. I believe if the user passes `sortKeySelector` which never returns nulls, they won't expect that `mergeUnion` would silently skip `keySelector`.
   
   Unfortunately, you do not declare the behavior with regards to the `null` values for `TSource` elements in `List<Enumerable<TSource>> sources`.
   
   ---
   
   I believe the way to fix this is to remove custom `null` handling and just pass the keys to `sortKeySelector` as is:
   
   ```java
       private int compare(TSource e1, TSource e2) {
         final TKey key1 = this.sortKeySelector.apply(e1);
         final TKey key2 = this.sortKeySelector.apply(e2);
         return this.sortComparator.compare(key1, key2);
       }
   ```
   

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       Currently you declare `sortKeySelector` as `Function1<TSource, TKey> sortKeySelector`.
   In other words, it **must** work for all `TSource` values no matter if `TSource` is null or not.
   
   On the other hand, if you implement custom `null` behavior, you basically augment user-provided `sortKeySelector`, and you basically forbid users to handle `nulls` differently.
   
   I believe this silent implementation detail is worse than a clear NPE in case the user-provided function fails to handle nulls.

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       > and (it's not so easy to guess) I believe a null in there can lead to a NPE.
   
   `linq4j` is a general-purpose library, and it must not depend on `PhysType` quirks

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       The problem with `e1 == null ? null` is that behavior is not documented, and it is not tested (there's no test which checks what happens in case user-provided `keySelector` wants to convert `null` to something non-`null`).
   
   `mergeUnion` method is already very generic (it has customization for `keySelector`, `comparator`), so I see no reasons why the method should have its own `null-handling`. It is generic enough to let users decide what they want to do with nulls.

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final RelCollation collation = traitSet.getCollation();
+    if (collation == null || collation == RelCollations.EMPTY) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+    for (RelNode input : inputs) {
+      final RelCollation inputCollation = input.getTraitSet().getCollation();
+      if (inputCollation == null || !inputCollation.satisfies(collation)) {
+        throw new IllegalArgumentException("EnumerableMergeUnion input does not satisfy collation "
+            + input);

Review comment:
       It would probably help if both collations were included to the exception message

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {

Review comment:
       Could you please move the checks to `matches` method or to the rule definition?
   
   Currently, the engine has to remember the rule match, execute it and only then it figures out the execution was useless (e.g. no collation).




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r568457009



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+
+  private static final Object NOT_INIT = new Object();
+  private static final Object FINISHED = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    this.inputs = new ArrayList<>(size);
+    for (Enumerable<TSource> source : sources) {
+      this.inputs.add(source.enumerator());
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.size(); i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs.get(i);
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      //noinspection unchecked
+      currentInputsValues[i] = (TSource) FINISHED;
+    } else {
+      currentInputsValues[i] = enumerator.current();
+    }
+  }
+
+  private boolean checkNotDuplicated(TSource value) {
+    if (processed == null) {
+      return true; // UNION ALL: no need to check duplicates
+    }
+
+    // check duplicates
+    @SuppressWarnings("dereference.of.nullable")
+    final EnumerableDefaults.Wrapped<TSource> wrapped = wrapper.apply(value);
+    if (!processed.contains(wrapped)) {
+      if (!processed.isEmpty()) {
+        // Since inputs are sorted, we do not need to keep in the set all the items that we
+        // have previously returned, just the ones with the same key, as soon as we see a new
+        // key, we can clear the set containing the items belonging to the previous key
+        final EnumerableDefaults.Wrapped<TSource> wrappedItemInSet = processed.iterator().next();

Review comment:
       Changed: keep `currentKeyInProcessedSet` variable




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r565329110



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+
+  private static final Object NOT_INIT = new Object();
+  private static final Object FINISHED = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    this.inputs = new ArrayList<>(size);
+    for (Enumerable<TSource> source : sources) {
+      this.inputs.add(source.enumerator());
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.size(); i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs.get(i);
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      //noinspection unchecked
+      currentInputsValues[i] = (TSource) FINISHED;
+    } else {
+      currentInputsValues[i] = enumerator.current();
+    }
+  }
+
+  private boolean checkNotDuplicated(TSource value) {
+    if (processed == null) {
+      return true; // UNION ALL: no need to check duplicates
+    }
+
+    // check duplicates
+    @SuppressWarnings("dereference.of.nullable")
+    final EnumerableDefaults.Wrapped<TSource> wrapped = wrapper.apply(value);
+    if (!processed.contains(wrapped)) {
+      if (!processed.isEmpty()) {
+        // Since inputs are sorted, we do not need to keep in the set all the items that we
+        // have previously returned, just the ones with the same key, as soon as we see a new
+        // key, we can clear the set containing the items belonging to the previous key
+        final EnumerableDefaults.Wrapped<TSource> wrappedItemInSet = processed.iterator().next();

Review comment:
       I guess `iterator().next()` would have noticeable overhead since it is invoked for every row.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562583185



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>

Review comment:
       Enumerator moved to separate class




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

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



[GitHub] [calcite] amaliujia commented on pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
amaliujia commented on pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#issuecomment-767852590


   @rubenada 
   
   thanks for addressing existing comments. Current implementation seems pretty good.  I will give another pass soon.


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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561926270



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

Review comment:
       In general,`suppressions` should be targeted. If you add `SuppressWarnings` to the full constructor, it might allow unexpected warnings to creep in.
   
   In other words, suppressions should be confined to their minimal scope.
   
   For instance,
   
   ```java
       @RequiresNonNull("inputs")
       @SuppressWarnings("method.invocation.invalid")
       private void initEnumerators(
           EnumerableDefaults.@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this
       ) {
         for (int i = 0; i < this.inputs.size(); i++) {
           this.moveEnumerator(i);
         }
       }
   ```




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r565910548



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+
+  private static final Object NOT_INIT = new Object();
+  private static final Object FINISHED = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    this.inputs = new ArrayList<>(size);
+    for (Enumerable<TSource> source : sources) {
+      this.inputs.add(source.enumerator());
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.size(); i++) {
+      moveEnumerator(i);

Review comment:
       It could be, but I'd lead towards leaving it as it is.
   `MergeUnionEnumerator` is "lazily created" by its parent Enumerable (`EnumerableDefaults#mergeUnion`), i.e. the enumerator is only created when/if `Enumerable#enumerator` method is called. At this point (when the enumerator of an enumerable is generated) I think it is fair to assume that the enumerator that has been created is going to be traversed, so the data would be accessed anyway.
   Thus I think that the refactoring that you propose would add more complexity to `moveNext` code without any gain for the vast majority of cases.
   This strategy of "lazy enumerator creation" in Enumerable + some initialization work in the enumerator construction is already in place by other algorithms, like `EnumerableDefaults#mergeJoin`.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561847791



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       If I don't annotate the TKey I get the following error:
   
   ```
   error: [argument.type.incompatible] incompatible argument for parameter arg0 of compare.
         return this.sortComparator.compare(key1, key2);
                                            ^
     found   : TKey[ extends @Initialized @Nullable Object super @Initialized @Nullable Void]
     required: TKey[ extends @Initialized @Nullable Object super @Initialized @NonNull Void]
   ```
   
   How should I annotate the TKey?




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560931469



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,

Review comment:
       It would be slightly better to have something like `Function1<? super TSource, ? extends TKey>` so users could pass method references with subtypes. I don't know if that combines with other types, however, in most cases `Function1` arguments should look like `Map#computeIfAbsent`:
   
   ```java
   public V computeIfAbsent(K key, Function<? super K, ? extends V> mappingFunction) {
   ```




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572026365



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       A slight note: it might be a good time to perform `currentInputsValues[i] = null` to release 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.

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562015869



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       The problem with `e1 == null ? null` is that behavior is not documented, and it is not tested (there's no test which checks what happens in case user-provided `keySelector` wants to convert `null` to something non-`null`).
   
   `mergeUnion` method is already very generic (it has customization for `keySelector`, `comparator`), so I see no reasons why the method should have its own `null-handling`. It is generic enough to let users decide what they want to do with nulls.




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572093994



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       I'm afraid statement-level annotations work for `variable declaration statements` only :-/
   So the WA could be to declare a variable that overrides the type for the cleanup purposes.
   
   ```java
   @SuppressWarnings("...")
   @Nullable TSouce[] inputsValues = currentInputsValues;
   inputsValues[i] = null;
   ```
   
   Or it could be method-level declaration (which I do not like either)
   Or it could be "extract the clanup to a separate method with suppression".




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561680662



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       It's a good idea, I'll work on 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.

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560523613



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,

Review comment:
       Are inputs ordering checked somewhere? E.g. check collations of inputs?




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560933462



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       The inputs are sorted, so you do not need `Set` here. A single `TSource previous` record is enough, then you could compare `previous` with a new entry to tell if the next one should be emitted or 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.

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561131278



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       collation is the first field (deptId), so $0.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561119182



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,

Review comment:
       Done.

##########
File path: core/src/test/java/org/apache/calcite/test/enumerable/EnumerableMergeUnionTest.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test.enumerable;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.JdbcTest;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Consumer;
+
+/**
+ * Unit test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableMergeUnion}.
+ */
+class EnumerableMergeUnionTest {
+
+  @Test void mergeUnionAllOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionAllOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByNameWithLimit() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name limit 3")
+        .explainContains("EnumerableLimit(fetch=[3])\n"

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.

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561920915



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,
+        Comparator<@Nullable TKey> sortComparator,
+        boolean all,
+        EqualityComparer<TSource> equalityComparer) {
+      this.sortKeySelector = sortKeySelector;
+      this.sortComparator = sortComparator;
+
+      if (all) {
+        this.processed = null;
+        this.wrapper = null;
+      } else {
+        this.processed = new HashSet<>();
+        this.wrapper = wrapperFor(equalityComparer);
+      }
+
+      final int size = sources.size();
+      this.inputs = new ArrayList<>(size);
+      for (Enumerable<TSource> source : sources) {
+        this.inputs.add(source.enumerator());
+      }
+
+      this.currentInputsValues = (TSource[]) new Object[size];
+      this.activeInputs = this.currentInputsValues.length;
+      this.currentValue = (TSource) NOT_INIT;
+
+      this.initEnumerators();
+    }
+
+    private void initEnumerators() {
+      for (int i = 0; i < this.inputs.size(); i++) {
+        this.moveEnumerator(i);
+      }
+    }
+
+    private void moveEnumerator(int i) {
+      final Enumerator<TSource> enumerator = this.inputs.get(i);
+      if (!enumerator.moveNext()) {
+        this.activeInputs--;
+        this.currentInputsValues[i] = (TSource) FINISHED;
+      } else {
+        this.currentInputsValues[i] = enumerator.current();
+      }
+    }
+
+    @SuppressWarnings("dereference.of.nullable")

Review comment:
       If you intend to suppress a single case only, please use `SuppressWarnings` for a local variable declaration like in
   
   ```java
   @SuppressWarnings("dereference.of.nullable")
   final Wrapped<TSource> wrapped = this.wrapper.apply(value);
   ```
   
   




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560943999



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final List<RelCollation> collations = traitSet.getTraits(RelCollationTraitDef.INSTANCE);
+    if (collations == null || collations.isEmpty()) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+  }
+
+  public static EnumerableMergeUnion create(RelCollation collation, List<RelNode> inputs,
+      boolean all) {
+    final RelOptCluster cluster = inputs.get(0).getCluster();
+    final RelTraitSet traitSet = cluster.traitSetOf(EnumerableConvention.INSTANCE).replace(
+        collation);
+    return new EnumerableMergeUnion(cluster, traitSet, inputs, all);
+  }
+
+  @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    // EnumerableUnion computes the whole union result in memory, whereas EnumerableMergeUnion
+    // does it step by step, loading only 1 item per input at a time, reflect this optimization
+    // on the cpu cost:
+    final RelOptCost cost = super.computeSelfCost(planner, mq);
+    if (cost == null) {
+      return null;
+    }
+    return planner.getCostFactory().makeCost(cost.getRows(), cost.getCpu() / 2, cost.getIo());

Review comment:
       So the only difference in computation complexity appears when `all=false`.
   `EnumerableUnion` needs to keep the full lookup table in memory (it needs to perform `hashSet.add` for every row), while `EnumerableMergeUnion` needs to track the previously emitted row and emit a new row in case the next one is different. `MergeUnion` still has to call `equals`, so the thing does not sound like `two times cheaper`.
   
   It looks like the `self` costs of both variations are more or less the same.




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560928313



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,
+        Comparator<@Nullable TKey> sortComparator,
+        boolean all,
+        EqualityComparer<TSource> equalityComparer) {
+      this.sortKeySelector = sortKeySelector;
+      this.sortComparator = sortComparator;
+
+      if (all) {
+        this.processed = null;
+        this.wrapper = null;
+      } else {
+        this.processed = new HashSet<>();
+        this.wrapper = wrapperFor(equalityComparer);
+      }
+
+      final int size = sources.size();
+      this.inputs = new ArrayList<>(size);
+      for (Enumerable<TSource> source : sources) {
+        this.inputs.add(source.enumerator());
+      }
+
+      this.currentInputsValues = (TSource[]) new Object[size];
+      this.activeInputs = this.currentInputsValues.length;
+      this.currentValue = (TSource) NOT_INIT;
+
+      this.initEnumerators();
+    }
+
+    private void initEnumerators() {
+      for (int i = 0; i < this.inputs.size(); i++) {
+        this.moveEnumerator(i);
+      }
+    }
+
+    private void moveEnumerator(int i) {
+      final Enumerator<TSource> enumerator = this.inputs.get(i);
+      if (!enumerator.moveNext()) {
+        this.activeInputs--;
+        this.currentInputsValues[i] = (TSource) FINISHED;
+      } else {
+        this.currentInputsValues[i] = enumerator.current();
+      }
+    }
+
+    @SuppressWarnings("dereference.of.nullable")

Review comment:
       Does it mean there's NPE hidden in the method somewhere?




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562466790



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,

Review comment:
       Everywhere in EnumerableDefaults we have `Function1<TSource, TKey>` I think we should keep it like this for consistency.
   Maybe your proposal could be addressed as separate evolution to be performed in bulk in all EnumerableDefaults methods. with this type of function pattern.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572060090



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       However, I am not being able to suppress the warning just in that specific line (the following suppression fails):
   ```
   @SuppressWarnings("assignment.type.incompatible")
   currentInputsValues[i] = null;
   ```
   But it works fine when I add the suppression to the whole method (which I'd like to avoid in order to keep the suppression on the specific context where it must be applied). Any ideas @vlsi ?
   




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

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



[GitHub] [calcite] rubenada commented on pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#issuecomment-766639159


   Thanks for your feedback @vlsi & @amaliujia .
   Do you think the PR is in a good shape to squash commits and proceed with the merge?


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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560925896



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>

Review comment:
       Should the class be moved to a standalone file? `EnumerableDefaults` is already big, and adding more classes here is sad :-/




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

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561588160



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);

Review comment:
       Maybe stupid question: why need to copy sort for each input (rather than use a copied Sort for all inputs)?

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);
+      inputs.add(
+          convert(newInput, newInput.getTraitSet().replace(EnumerableConvention.INSTANCE)));
+    }
+
+    RelNode result = EnumerableMergeUnion.create(sort.getCollation(), inputs, union.all);
+
+    // If Sort contained a LIMIT / OFFSET, then put it back as an EnumerableLimit.
+    // The output of the MergeUnion is already sorted, so we do not need a sort anymore.
+    if (sort.offset != null || sort.fetch != null) {
+      result = EnumerableLimit.create(result, sort.offset, sort.fetch);

Review comment:
       I think you will need to set the collation in  EnumerableLimit? Basically the collation need to be passed through?

##########
File path: core/src/test/java/org/apache/calcite/test/enumerable/EnumerableMergeUnionTest.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test.enumerable;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.JdbcTest;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Consumer;
+
+/**
+ * Unit test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableMergeUnion}.
+ */
+class EnumerableMergeUnionTest {
+
+  @Test void mergeUnionAllOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionAllOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByNameWithLimit() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name limit 3")
+        .explainContains("EnumerableLimit(fetch=[3])\n"

Review comment:
       This is a good point!




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572048044



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       That depends.
   a) The declaration can to be `@Nullable TSource[] currentInputsValues`. That would need to handle `null` in all use cases
   b) `null` store could be suppressed




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561933483



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       ```java
       private int compare(TSource e1, TSource e2) {
         final TKey key1 = e1 == null ? null : this.sortKeySelector.apply(e1);
         final TKey key2 = e2 == null ? null : this.sortKeySelector.apply(e2);
         return this.sortComparator.compare(key1, key2);
       }
   ```
   
   Ok, what does the error mean here?
   It says that the comparison key can become `null` even in the case `sortKeySelector` never returns nulls. I believe if the user passes `sortKeySelector` which never returns nulls, they won't expect that `mergeUnion` would silently skip `keySelector`.
   
   Unfortunately, you do not declare the behavior with regards to the `null` values for `TSource` elements in `List<Enumerable<TSource>> sources`.
   
   ---
   
   I believe the way to fix this is to remove custom `null` handling and just pass the keys to `sortKeySelector` as is:
   
   ```java
       private int compare(TSource e1, TSource e2) {
         final TKey key1 = this.sortKeySelector.apply(e1);
         final TKey key2 = this.sortKeySelector.apply(e2);
         return this.sortComparator.compare(key1, key2);
       }
   ```
   




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572054683



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       I'd lead towards `b` in this particular case




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561942632



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>

Review comment:
       Agree, I'll move to a separate file once the other discussions get resolved.




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r565272219



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+
+  private static final Object NOT_INIT = new Object();
+  private static final Object FINISHED = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    this.inputs = new ArrayList<>(size);
+    for (Enumerable<TSource> source : sources) {
+      this.inputs.add(source.enumerator());
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.size(); i++) {
+      moveEnumerator(i);

Review comment:
       Should it be called within `moveNext` to avoid early access to the data?




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

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



[GitHub] [calcite] vlsi commented on pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
vlsi commented on pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#issuecomment-775133262


   @rubenada , thanks. It looks good to me. It might be worth to set `currentInputsValues[i]` to `null` as an input drains 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.

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560927169



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

Review comment:
       Am I right this is only for `initEnumerators`?
   If so, it would be great to move the suppression to the relevant method only + add `@UnderInitialization this` parameter to `initEnumerators`
   




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

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



[GitHub] [calcite] rubenada commented on pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#issuecomment-775043865


   @vlsi I have addressed your latest comments, could you please take a final look and let me know what you think? I'd like to include this feature in the upcoming release 1.27.


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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572093994



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       I'm afraid statement-level suppression works for `variable declaration statements` only :-/
   So the WA could be to declare a variable that overrides the type for the cleanup purposes.
   
   ```java
   @SuppressWarnings("...")
   @Nullable TSouce inputsValues = currentInputsValues;
   inputsValues[i] = null;
   ```
   
   Or it could be method-level declaration (which I do not like either)
   Or it could be "extract the clanup to a separate method with suppression".

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       I'm afraid statement-level annotations work for `variable declaration statements` only :-/
   So the WA could be to declare a variable that overrides the type for the cleanup purposes.
   
   ```java
   @SuppressWarnings("...")
   @Nullable TSouce inputsValues = currentInputsValues;
   inputsValues[i] = null;
   ```
   
   Or it could be method-level declaration (which I do not like either)
   Or it could be "extract the clanup to a separate method with suppression".




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561760327



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       Optimization committed




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

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



[GitHub] [calcite] rubenada commented on pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#issuecomment-775138051


   Thanks @vlsi . I'll apply your latest suggestion. If nobody presents any other comment, I'll merge the PR in the next 24/48 h.


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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562023629



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       Change committed. Thanks for your input @vlsi 




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560850407



##########
File path: core/src/test/java/org/apache/calcite/test/enumerable/EnumerableMergeUnionTest.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test.enumerable;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.JdbcTest;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Consumer;
+
+/**
+ * Unit test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableMergeUnion}.
+ */
+class EnumerableMergeUnionTest {
+
+  @Test void mergeUnionAllOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionAllOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByNameWithLimit() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name limit 3")
+        .explainContains("EnumerableLimit(fetch=[3])\n"

Review comment:
       That would work, correct me if I am wrong, only if we have a "pure LIMIT". If we have a LIMIT + OFFSET (or just OFFSET), it cannot be pushed into the downstream nodes.




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561129102



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       > Input1 = Emp(10, Bob), Emp(20, Joe), Emp(20, Greg)
   
   What is the collation?
   




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560796382



##########
File path: core/src/test/java/org/apache/calcite/test/enumerable/EnumerableMergeUnionTest.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test.enumerable;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.JdbcTest;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Consumer;
+
+/**
+ * Unit test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableMergeUnion}.
+ */
+class EnumerableMergeUnionTest {
+
+  @Test void mergeUnionAllOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionAllOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByNameWithLimit() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name limit 3")
+        .explainContains("EnumerableLimit(fetch=[3])\n"

Review comment:
       Can you please propagate limit to the downstream nodes?
   
   In other words, if you want 3 rows from the union, then you could limit each branch with 3 so they can use limited sort rather than sorting of the full result.
   
   WDYT?




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

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561588160



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);

Review comment:
       Maybe stupid question: why need to copy sort for each input (rather than use a copied Sort for all inputs)?

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);
+      inputs.add(
+          convert(newInput, newInput.getTraitSet().replace(EnumerableConvention.INSTANCE)));
+    }
+
+    RelNode result = EnumerableMergeUnion.create(sort.getCollation(), inputs, union.all);
+
+    // If Sort contained a LIMIT / OFFSET, then put it back as an EnumerableLimit.
+    // The output of the MergeUnion is already sorted, so we do not need a sort anymore.
+    if (sort.offset != null || sort.fetch != null) {
+      result = EnumerableLimit.create(result, sort.offset, sort.fetch);

Review comment:
       I think you will need to set the collation in  EnumerableLimit? Basically the collation need to be passed through?

##########
File path: core/src/test/java/org/apache/calcite/test/enumerable/EnumerableMergeUnionTest.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test.enumerable;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.JdbcTest;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Consumer;
+
+/**
+ * Unit test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableMergeUnion}.
+ */
+class EnumerableMergeUnionTest {
+
+  @Test void mergeUnionAllOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionAllOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByNameWithLimit() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name limit 3")
+        .explainContains("EnumerableLimit(fetch=[3])\n"

Review comment:
       This is a good point!

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);

Review comment:
       Yes that's the good point :)




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562005945



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")

Review comment:
       Done

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

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.

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r568456474



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+
+  private static final Object NOT_INIT = new Object();
+  private static final Object FINISHED = new Object();

Review comment:
       changed: source state now tracked via boolean array `inputsFinished`

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;

Review comment:
       Changed: everything as array

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+
+  private static final Object NOT_INIT = new Object();
+  private static final Object FINISHED = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    this.inputs = new ArrayList<>(size);
+    for (Enumerable<TSource> source : sources) {
+      this.inputs.add(source.enumerator());
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.size(); i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs.get(i);
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      //noinspection unchecked
+      currentInputsValues[i] = (TSource) FINISHED;
+    } else {
+      currentInputsValues[i] = enumerator.current();
+    }
+  }
+
+  private boolean checkNotDuplicated(TSource value) {
+    if (processed == null) {
+      return true; // UNION ALL: no need to check duplicates
+    }
+
+    // check duplicates
+    @SuppressWarnings("dereference.of.nullable")
+    final EnumerableDefaults.Wrapped<TSource> wrapped = wrapper.apply(value);
+    if (!processed.contains(wrapped)) {
+      if (!processed.isEmpty()) {
+        // Since inputs are sorted, we do not need to keep in the set all the items that we
+        // have previously returned, just the ones with the same key, as soon as we see a new
+        // key, we can clear the set containing the items belonging to the previous key
+        final EnumerableDefaults.Wrapped<TSource> wrappedItemInSet = processed.iterator().next();

Review comment:
       Changed: keep `currentKeyInProcessedSet` variable




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560882317



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final List<RelCollation> collations = traitSet.getTraits(RelCollationTraitDef.INSTANCE);
+    if (collations == null || collations.isEmpty()) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+  }
+
+  public static EnumerableMergeUnion create(RelCollation collation, List<RelNode> inputs,
+      boolean all) {
+    final RelOptCluster cluster = inputs.get(0).getCluster();
+    final RelTraitSet traitSet = cluster.traitSetOf(EnumerableConvention.INSTANCE).replace(
+        collation);
+    return new EnumerableMergeUnion(cluster, traitSet, inputs, all);
+  }
+
+  @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    // EnumerableUnion computes the whole union result in memory, whereas EnumerableMergeUnion

Review comment:
       `EnumerableUnion` does not keep all the results in memory in case `all=true` (==`UNION ALL`)




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561015434



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final List<RelCollation> collations = traitSet.getTraits(RelCollationTraitDef.INSTANCE);
+    if (collations == null || collations.isEmpty()) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+  }
+
+  public static EnumerableMergeUnion create(RelCollation collation, List<RelNode> inputs,
+      boolean all) {
+    final RelOptCluster cluster = inputs.get(0).getCluster();
+    final RelTraitSet traitSet = cluster.traitSetOf(EnumerableConvention.INSTANCE).replace(
+        collation);
+    return new EnumerableMergeUnion(cluster, traitSet, inputs, all);
+  }
+
+  @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    // EnumerableUnion computes the whole union result in memory, whereas EnumerableMergeUnion
+    // does it step by step, loading only 1 item per input at a time, reflect this optimization
+    // on the cpu cost:
+    final RelOptCost cost = super.computeSelfCost(planner, mq);
+    if (cost == null) {
+      return null;
+    }
+    return planner.getCostFactory().makeCost(cost.getRows(), cost.getCpu() / 2, cost.getIo());

Review comment:
       Ok, I'll remove this overridden method. Let EnumerableMergeUnion have by default the same cost as EnumerableUnion.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562023312



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,

Review comment:
       Reviewed




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561847791



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       If I don't annotate the TKey I get the following error:
   ````
   error: [argument.type.incompatible] incompatible argument for parameter arg0 of compare.
         return this.sortComparator.compare(key1, key2);
                                            ^
     found   : TKey[ extends @Initialized @Nullable Object super @Initialized @Nullable Void]
     required: TKey[ extends @Initialized @Nullable Object super @Initialized @NonNull Void]
   ```
   How should I annotate the TKey?




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572048044



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       That depends.
   a) The declaration can to be `@Nullable TSource[] currentInputsValues`. That would need to handle `null` in all use cases
   b) `null` store could be suppressed like "ok, `TSource` does not always permit null, however, we would never use `currentInputsValues[i]` since `inputsFinished[i] = true`"




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561681109



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);

Review comment:
       Each Sort has a different input (each input from the union)




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572031381



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       Good point. I'll apply 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.

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560879907



##########
File path: core/src/test/java/org/apache/calcite/test/enumerable/EnumerableMergeUnionTest.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test.enumerable;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.JdbcTest;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Consumer;
+
+/**
+ * Unit test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableMergeUnion}.
+ */
+class EnumerableMergeUnionTest {
+
+  @Test void mergeUnionAllOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionOrderByEmpid() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by empid")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$0], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=1; name=Bill",
+            "empid=6; name=Guy",
+            "empid=10; name=Gabriel",
+            "empid=12; name=Paul",
+            "empid=29; name=Anibal",
+            "empid=40; name=Emmanuel",
+            "empid=45; name=Pascal");
+  }
+
+  @Test void mergeUnionAllOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union all select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[true])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByName() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name")
+        .explainContains("EnumerableMergeUnion(all=[false])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['G%'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n"
+            + "  EnumerableSort(sort0=[$1], dir0=[ASC])\n"
+            + "    EnumerableCalc(expr#0..4=[{inputs}], expr#5=['%l'], expr#6=[LIKE($t2, $t5)], empid=[$t0], name=[$t2], $condition=[$t6])\n"
+            + "      EnumerableTableScan(table=[[s, emps]])\n")
+        .returnsOrdered(
+            "empid=29; name=Anibal",
+            "empid=1; name=Bill",
+            "empid=40; name=Emmanuel",
+            "empid=10; name=Gabriel",
+            "empid=6; name=Guy",
+            "empid=45; name=Pascal",
+            "empid=12; name=Paul");
+  }
+
+  @Test void mergeUnionOrderByNameWithLimit() {
+    tester(false,
+        new JdbcTest.HrSchemaBig(),
+        "select * from (select empid, name from emps where name like 'G%' union select empid, name from emps where name like '%l') order by name limit 3")
+        .explainContains("EnumerableLimit(fetch=[3])\n"

Review comment:
       `OFFSET x LIMIT y` can be transformed to the downstream `LIMIT x+y`




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

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



[GitHub] [calcite] rubenada merged pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada merged pull request #2329:
URL: https://github.com/apache/calcite/pull/2329


   


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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561166786



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       On the other hand, the set can be invalidated as the collation key changes which would reduce memory consumption. WDYT?




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561852442



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

Review comment:
       Yes, this is about `initEnumerators`:
   ```
   error: [method.invocation.invalid] call to initEnumerators() not allowed on the given receiver.
         this.initEnumerators();
                             ^
     found   : @UnderInitialization(org.apache.calcite.linq4j.EnumerableDefaults.MergeUnionEnumerator.class) @NonNull MergeUnionEnumerator
     required: @Initialized @NonNull MergeUnionEnumerator
   ```
   Sorry, I did not understand your problem,  how should I fix it?

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

Review comment:
       Yes, this is about `initEnumerators`:
   ```
   error: [method.invocation.invalid] call to initEnumerators() not allowed on the given receiver.
         this.initEnumerators();
                             ^
     found   : @UnderInitialization(org.apache.calcite.linq4j.EnumerableDefaults.MergeUnionEnumerator.class) @NonNull MergeUnionEnumerator
     required: @Initialized @NonNull MergeUnionEnumerator
   ```
   Sorry, I did not understand your suggestion,  how should I fix 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.

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561138796



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       Oh, so you mean that `sort-merge union` keeps the common collation while merging the results, however, it should eliminate duplicates no matter the order.
   
   Then the `set` is indeed required. However, the set can be skipped in case all the columns participate in the collation (e.g. `order by id, 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.

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



[GitHub] [calcite] amaliujia commented on pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
amaliujia commented on pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#issuecomment-768045556


   LGTM 


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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560940304



##########
File path: core/src/test/java/org/apache/calcite/test/enumerable/EnumerableMergeUnionTest.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test.enumerable;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.JdbcTest;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Consumer;
+
+/**
+ * Unit test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableMergeUnion}.
+ */
+class EnumerableMergeUnionTest {

Review comment:
       Just wondering: does it make sense to add tests for `single-column` unions as well?
   I guess it might trigger slightly different `physType`.




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562037243



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {

Review comment:
       Could you please move the checks to `matches` method or to the rule definition?
   
   Currently, the engine has to remember the rule match, execute it and only then it figures out the execution was useless (e.g. no collation).




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560921441



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final List<RelCollation> collations = traitSet.getTraits(RelCollationTraitDef.INSTANCE);
+    if (collations == null || collations.isEmpty()) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+  }
+
+  public static EnumerableMergeUnion create(RelCollation collation, List<RelNode> inputs,
+      boolean all) {
+    final RelOptCluster cluster = inputs.get(0).getCluster();
+    final RelTraitSet traitSet = cluster.traitSetOf(EnumerableConvention.INSTANCE).replace(
+        collation);
+    return new EnumerableMergeUnion(cluster, traitSet, inputs, all);
+  }
+
+  @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    // EnumerableUnion computes the whole union result in memory, whereas EnumerableMergeUnion
+    // does it step by step, loading only 1 item per input at a time, reflect this optimization
+    // on the cpu cost:
+    final RelOptCost cost = super.computeSelfCost(planner, mq);
+    if (cost == null) {
+      return null;
+    }
+    return planner.getCostFactory().makeCost(cost.getRows(), cost.getCpu() / 2, cost.getIo());

Review comment:
       I believe this `/2` should not be applied for `all=true` case.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r568456474



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+
+  private static final Object NOT_INIT = new Object();
+  private static final Object FINISHED = new Object();

Review comment:
       changed: source state now tracked via boolean array `inputsFinished`

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;

Review comment:
       Changed: everything as array




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561131278



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       collation is the first field (deptId), so $0:
   UNION, collation[0]
     Input1, collation[0] = Emp(10, Bob), Emp(20, Joe), Emp(20, Greg)
     Input2, collation[0] = Emp(15, Laura), Emp(20, Joe)




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

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



[GitHub] [calcite] amaliujia commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562121640



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);

Review comment:
       Yes that's the good point :)




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560929944



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,

Review comment:
       Why do you force clients to use `nullable` type for the function?
   I guess `Function1<TSource, TKey>` could be enough here.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560946645



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       You're right, I will work in the `TSource previous` approach




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562055776



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {

Review comment:
       matches method overridden with checks

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final RelCollation collation = traitSet.getCollation();
+    if (collation == null || collation == RelCollations.EMPTY) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+    for (RelNode input : inputs) {
+      final RelCollation inputCollation = input.getTraitSet().getCollation();
+      if (inputCollation == null || !inputCollation.satisfies(collation)) {
+        throw new IllegalArgumentException("EnumerableMergeUnion input does not satisfy collation "
+            + input);

Review comment:
       exception message improved




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561119717



##########
File path: core/src/test/java/org/apache/calcite/test/enumerable/EnumerableMergeUnionTest.java
##########
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.test.enumerable;
+
+import org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.calcite.adapter.java.ReflectiveSchema;
+import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.config.Lex;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.test.CalciteAssert;
+import org.apache.calcite.test.JdbcTest;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Consumer;
+
+/**
+ * Unit test for
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableMergeUnion}.
+ */
+class EnumerableMergeUnionTest {

Review comment:
       Added new tests for single column




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561680662



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       It's a good idea, I'll work on that.

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);

Review comment:
       Each Sort has a different input (each input from the union)

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {
+      return;
+    }
+
+    final Union union = call.rel(1);
+    final int unionInputsSize = union.getInputs().size();
+    if (unionInputsSize < 2) {
+      return;
+    }
+
+    // Push down sort limit, if possible.
+    RexNode inputFetch = null;
+    if (sort.fetch != null) {
+      if (sort.offset == null) {
+        inputFetch = sort.fetch;
+      } else if (sort.fetch instanceof RexLiteral && sort.offset instanceof RexLiteral) {
+        inputFetch = call.builder().literal(
+            RexLiteral.intValue(sort.fetch) + RexLiteral.intValue(sort.offset));
+      }
+    }
+
+    final List<RelNode> inputs = new ArrayList<>(unionInputsSize);
+    for (RelNode input : union.getInputs()) {
+      final RelNode newInput = sort.copy(sort.getTraitSet(), input, collation, null, inputFetch);
+      inputs.add(
+          convert(newInput, newInput.getTraitSet().replace(EnumerableConvention.INSTANCE)));
+    }
+
+    RelNode result = EnumerableMergeUnion.create(sort.getCollation(), inputs, union.all);
+
+    // If Sort contained a LIMIT / OFFSET, then put it back as an EnumerableLimit.
+    // The output of the MergeUnion is already sorted, so we do not need a sort anymore.
+    if (sort.offset != null || sort.fetch != null) {
+      result = EnumerableLimit.create(result, sort.offset, sort.fetch);

Review comment:
       This is already taken care of by E`numerableLimit#create` method, which calls `RelMdCollation.limit(mq, input)` (which just takes the collation from the limit's input).

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;

Review comment:
       Optimization committed

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       If I don't annotate the TKey I get the following error:
   ````
   error: [argument.type.incompatible] incompatible argument for parameter arg0 of compare.
         return this.sortComparator.compare(key1, key2);
                                            ^
     found   : TKey[ extends @Initialized @Nullable Object super @Initialized @Nullable Void]
     required: TKey[ extends @Initialized @Nullable Object super @Initialized @NonNull Void]
   ```
   How should I annotate the TKey?

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       If I don't annotate the TKey I get the following error:
   
   ```
   error: [argument.type.incompatible] incompatible argument for parameter arg0 of compare.
         return this.sortComparator.compare(key1, key2);
                                            ^
     found   : TKey[ extends @Initialized @Nullable Object super @Initialized @Nullable Void]
     required: TKey[ extends @Initialized @Nullable Object super @Initialized @NonNull Void]
   ```
   
   How should I annotate the TKey?

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,
+        Comparator<@Nullable TKey> sortComparator,
+        boolean all,
+        EqualityComparer<TSource> equalityComparer) {
+      this.sortKeySelector = sortKeySelector;
+      this.sortComparator = sortComparator;
+
+      if (all) {
+        this.processed = null;
+        this.wrapper = null;
+      } else {
+        this.processed = new HashSet<>();
+        this.wrapper = wrapperFor(equalityComparer);
+      }
+
+      final int size = sources.size();
+      this.inputs = new ArrayList<>(size);
+      for (Enumerable<TSource> source : sources) {
+        this.inputs.add(source.enumerator());
+      }
+
+      this.currentInputsValues = (TSource[]) new Object[size];
+      this.activeInputs = this.currentInputsValues.length;
+      this.currentValue = (TSource) NOT_INIT;
+
+      this.initEnumerators();
+    }
+
+    private void initEnumerators() {
+      for (int i = 0; i < this.inputs.size(); i++) {
+        this.moveEnumerator(i);
+      }
+    }
+
+    private void moveEnumerator(int i) {
+      final Enumerator<TSource> enumerator = this.inputs.get(i);
+      if (!enumerator.moveNext()) {
+        this.activeInputs--;
+        this.currentInputsValues[i] = (TSource) FINISHED;
+      } else {
+        this.currentInputsValues[i] = enumerator.current();
+      }
+    }
+
+    @SuppressWarnings("dereference.of.nullable")

Review comment:
       No. CheckerFramework flags `wrapper.apply` as a possible NPE because `wrapper` can be theoretically null. However, `wrapper`  is null only when `processed` is null (and vice-versa), and since the method checks for `processed` nullability at the beginning, we can never get a NPE on  `wrapper.apply` 

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

Review comment:
       Yes, this is about `initEnumerators`:
   ```
   error: [method.invocation.invalid] call to initEnumerators() not allowed on the given receiver.
         this.initEnumerators();
                             ^
     found   : @UnderInitialization(org.apache.calcite.linq4j.EnumerableDefaults.MergeUnionEnumerator.class) @NonNull MergeUnionEnumerator
     required: @Initialized @NonNull MergeUnionEnumerator
   ```
   Sorry, I did not understand your problem,  how should I fix it?

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

Review comment:
       Yes, this is about `initEnumerators`:
   ```
   error: [method.invocation.invalid] call to initEnumerators() not allowed on the given receiver.
         this.initEnumerators();
                             ^
     found   : @UnderInitialization(org.apache.calcite.linq4j.EnumerableDefaults.MergeUnionEnumerator.class) @NonNull MergeUnionEnumerator
     required: @Initialized @NonNull MergeUnionEnumerator
   ```
   Sorry, I did not understand your suggestion,  how should I fix it?

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>

Review comment:
       Agree, I'll move to a separate file once the other discussions get resolved.

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       I fear that you suggestion might trigger a NPE inside `sortKeySelector.apply` if we pass a null element e1/e2.
   BTW `sortComparator.compare` should be able to handle (in some cases) null keys (e.g. when we have nulls first or nulls last, and the comparator must handle nulls accordingly when compared to a non-null value).

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       I understand. The complication is that this `sortKeySelector` is / can be created by dynamic code (`PhysTpe#generateCollationKey`), and (it's not so easy to guess) I believe a null in there can lead to a NPE.
   I have committed a new workaround based on `@SuppressWarnings("argument.type.incompatible")`

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")

Review comment:
       Done

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")

Review comment:
       Done

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,
+        Comparator<@Nullable TKey> sortComparator,
+        boolean all,
+        EqualityComparer<TSource> equalityComparer) {
+      this.sortKeySelector = sortKeySelector;
+      this.sortComparator = sortComparator;
+
+      if (all) {
+        this.processed = null;
+        this.wrapper = null;
+      } else {
+        this.processed = new HashSet<>();
+        this.wrapper = wrapperFor(equalityComparer);
+      }
+
+      final int size = sources.size();
+      this.inputs = new ArrayList<>(size);
+      for (Enumerable<TSource> source : sources) {
+        this.inputs.add(source.enumerator());
+      }
+
+      this.currentInputsValues = (TSource[]) new Object[size];
+      this.activeInputs = this.currentInputsValues.length;
+      this.currentValue = (TSource) NOT_INIT;
+
+      this.initEnumerators();
+    }
+
+    private void initEnumerators() {
+      for (int i = 0; i < this.inputs.size(); i++) {
+        this.moveEnumerator(i);
+      }
+    }
+
+    private void moveEnumerator(int i) {
+      final Enumerator<TSource> enumerator = this.inputs.get(i);
+      if (!enumerator.moveNext()) {
+        this.activeInputs--;
+        this.currentInputsValues[i] = (TSource) FINISHED;
+      } else {
+        this.currentInputsValues[i] = enumerator.current();
+      }
+    }
+
+    @SuppressWarnings("dereference.of.nullable")

Review comment:
       Done

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       You're right, in other parts of `EnumerableDefaults` using `Function1<TSource, TKey> keySelector` there is no null check before calling `keySelector.apply`, I'll remove the null handling here.

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,

Review comment:
       Reviewed

##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       Change committed. Thanks for your input @vlsi 

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnionRule.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalSort} on top of a
+ * {@link org.apache.calcite.rel.logical.LogicalUnion} into a {@link EnumerableMergeUnion}.
+ *
+ * @see EnumerableRules#ENUMERABLE_MERGE_UNION_RULE
+ */
+public class EnumerableMergeUnionRule extends RelRule<EnumerableMergeUnionRule.Config> {
+
+  /** Rule configuration. */
+  public interface Config extends RelRule.Config {
+    Config DEFAULT_CONFIG = EMPTY.withDescription("EnumerableMergeUnionRule").withOperandSupplier(
+        b0 -> b0.operand(LogicalSort.class).oneInput(
+            b1 -> b1.operand(LogicalUnion.class).anyInputs())).as(Config.class);
+
+    @Override default EnumerableMergeUnionRule toRule() {
+      return new EnumerableMergeUnionRule(this);
+    }
+  }
+
+  public EnumerableMergeUnionRule(Config config) {
+    super(config);
+  }
+
+  @Override public void onMatch(RelOptRuleCall call) {
+    final Sort sort = call.rel(0);
+    final RelCollation collation = sort.getCollation();
+    if (collation == null || collation.getFieldCollations().isEmpty()) {

Review comment:
       matches method overridden with checks

##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final RelCollation collation = traitSet.getCollation();
+    if (collation == null || collation == RelCollations.EMPTY) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+    for (RelNode input : inputs) {
+      final RelCollation inputCollation = input.getTraitSet().getCollation();
+      if (inputCollation == null || !inputCollation.satisfies(collation)) {
+        throw new IllegalArgumentException("EnumerableMergeUnion input does not satisfy collation "
+            + input);

Review comment:
       exception message improved




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r565273524



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+
+  private static final Object NOT_INIT = new Object();
+  private static final Object FINISHED = new Object();

Review comment:
       This is a clever use of unchecked assignments, however, I would prefer a separate `enum SourceState { NOT_INITIALIZED, IN_PROGRESS, FINISHED; }`  + `SourceState[] currentInputSteates` array.
   
   It would separate data and metadata, and it would avoid accidental "helper object" leaks to the caller code.




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r565275001



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final List<Enumerator<TSource>> inputs;
+  private final TSource[] currentInputsValues;

Review comment:
       Should the data types be aligned?
   For instance, `everything stored as arrays` or `everything stored as lists`.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562005253



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       I understand. The complication is that this `sortKeySelector` is / can be created by dynamic code (`PhysTpe#generateCollationKey`), and (it's not so easy to guess) I believe a null in there can lead to a NPE.
   I have committed a new workaround based on `@SuppressWarnings("argument.type.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.

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561119400



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final List<RelCollation> collations = traitSet.getTraits(RelCollationTraitDef.INSTANCE);
+    if (collations == null || collations.isEmpty()) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+  }
+
+  public static EnumerableMergeUnion create(RelCollation collation, List<RelNode> inputs,
+      boolean all) {
+    final RelOptCluster cluster = inputs.get(0).getCluster();
+    final RelTraitSet traitSet = cluster.traitSetOf(EnumerableConvention.INSTANCE).replace(
+        collation);
+    return new EnumerableMergeUnion(cluster, traitSet, inputs, all);
+  }
+
+  @Override public @Nullable RelOptCost computeSelfCost(RelOptPlanner planner,
+      RelMetadataQuery mq) {
+    // EnumerableUnion computes the whole union result in memory, whereas EnumerableMergeUnion
+    // does it step by step, loading only 1 item per input at a time, reflect this optimization
+    // on the cpu cost:
+    final RelOptCost cost = super.computeSelfCost(planner, mq);
+    if (cost == null) {
+      return null;
+    }
+    return planner.getCostFactory().makeCost(cost.getRows(), cost.getCpu() / 2, cost.getIo());

Review comment:
       Method 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.

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561851275



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, @Nullable TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    return new AbstractEnumerable<TSource>() {
+      @Override public Enumerator<TSource> enumerator() {
+        return new MergeUnionEnumerator<>(
+            sources,
+            sortKeySelector,
+            sortComparator,
+            all,
+            equalityComparer);
+      }
+    };
+  }
+
+  /**
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  @SuppressWarnings("unchecked")
+  private static class MergeUnionEnumerator<TSource, @Nullable TKey>
+      implements Enumerator<TSource> {
+    private final List<Enumerator<TSource>> inputs;
+    private final TSource[] currentInputsValues;
+    private final Function1<TSource, @Nullable TKey> sortKeySelector;
+    private final Comparator<@Nullable TKey> sortComparator;
+    private TSource currentValue;
+    private int activeInputs;
+
+    // Set to control duplicates, only used if "all" is false
+    private final @Nullable Set<Wrapped<TSource>> processed;
+    private final @Nullable Function1<TSource, Wrapped<TSource>> wrapper;
+
+    private static final Object NOT_INIT = new Object();
+    private static final Object FINISHED = new Object();
+
+    @SuppressWarnings("method.invocation.invalid")
+    private MergeUnionEnumerator(
+        List<Enumerable<TSource>> sources,
+        Function1<TSource, @Nullable TKey> sortKeySelector,
+        Comparator<@Nullable TKey> sortComparator,
+        boolean all,
+        EqualityComparer<TSource> equalityComparer) {
+      this.sortKeySelector = sortKeySelector;
+      this.sortComparator = sortComparator;
+
+      if (all) {
+        this.processed = null;
+        this.wrapper = null;
+      } else {
+        this.processed = new HashSet<>();
+        this.wrapper = wrapperFor(equalityComparer);
+      }
+
+      final int size = sources.size();
+      this.inputs = new ArrayList<>(size);
+      for (Enumerable<TSource> source : sources) {
+        this.inputs.add(source.enumerator());
+      }
+
+      this.currentInputsValues = (TSource[]) new Object[size];
+      this.activeInputs = this.currentInputsValues.length;
+      this.currentValue = (TSource) NOT_INIT;
+
+      this.initEnumerators();
+    }
+
+    private void initEnumerators() {
+      for (int i = 0; i < this.inputs.size(); i++) {
+        this.moveEnumerator(i);
+      }
+    }
+
+    private void moveEnumerator(int i) {
+      final Enumerator<TSource> enumerator = this.inputs.get(i);
+      if (!enumerator.moveNext()) {
+        this.activeInputs--;
+        this.currentInputsValues[i] = (TSource) FINISHED;
+      } else {
+        this.currentInputsValues[i] = enumerator.current();
+      }
+    }
+
+    @SuppressWarnings("dereference.of.nullable")

Review comment:
       No. CheckerFramework flags `wrapper.apply` as a possible NPE because `wrapper` can be theoretically null. However, `wrapper`  is null only when `processed` is null (and vice-versa), and since the method checks for `processed` nullability at the beginning, we can never get a NPE on  `wrapper.apply` 




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r560782371



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,

Review comment:
       You're right, I should add in the constructor the verification that all inputs satisfy the collation.




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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r561950919



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       Currently you declare `sortKeySelector` as `Function1<TSource, TKey> sortKeySelector`.
   In other words, it **must** work for all `TSource` values no matter if `TSource` is null or not.
   
   On the other hand, if you implement custom `null` behavior, you basically augment user-provided `sortKeySelector`, and you basically forbid users to handle `nulls` differently.
   
   I believe this silent implementation detail is worse than a clear NPE in case the user-provided function fails to handle nulls.




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

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



[GitHub] [calcite] rubenada commented on pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#issuecomment-766639159


   Thanks for your feedback @vlsi & @amaliujia .
   Do you think the PR is in a good shape to squash commits and proceed with the merge?


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

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



[GitHub] [calcite] vlsi commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
vlsi commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562034886



##########
File path: core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableMergeUnion.java
##########
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Union} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}.
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order. */
+public class EnumerableMergeUnion extends EnumerableUnion {
+
+  protected EnumerableMergeUnion(RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    super(cluster, traitSet, inputs, all);
+    final RelCollation collation = traitSet.getCollation();
+    if (collation == null || collation == RelCollations.EMPTY) {
+      throw new IllegalArgumentException("EnumerableMergeUnion with no collation");
+    }
+    for (RelNode input : inputs) {
+      final RelCollation inputCollation = input.getTraitSet().getCollation();
+      if (inputCollation == null || !inputCollation.satisfies(collation)) {
+        throw new IllegalArgumentException("EnumerableMergeUnion input does not satisfy collation "
+            + input);

Review comment:
       It would probably help if both collations were included to the exception message




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add MergeUnion operator in Enumerable convention

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r572164751



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/MergeUnionEnumerator.java
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.linq4j;
+
+import org.apache.calcite.linq4j.function.EqualityComparer;
+import org.apache.calcite.linq4j.function.Function1;
+
+import org.checkerframework.checker.initialization.qual.UnknownInitialization;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.checkerframework.checker.nullness.qual.RequiresNonNull;
+
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+
+/**
+ * Performs a union (or union all) of all its inputs (which must be already sorted),
+ * respecting the order.
+ * @param <TSource> record type
+ * @param <TKey> sort key
+ */
+final class MergeUnionEnumerator<TSource, TKey> implements Enumerator<TSource> {
+  private final Enumerator<TSource>[] inputs;
+  private final TSource[] currentInputsValues;
+  private final boolean[] inputsFinished;
+  private final Function1<TSource, TKey> sortKeySelector;
+  private final Comparator<TKey> sortComparator;
+  private TSource currentValue;
+  private int activeInputs;
+
+  // Set to control duplicates, only used if "all" is false
+  private final @Nullable Set<EnumerableDefaults.Wrapped<TSource>> processed;
+  private final @Nullable Function1<TSource, EnumerableDefaults.Wrapped<TSource>> wrapper;
+  private @Nullable TKey currentKeyInProcessedSet;
+
+  private static final Object NOT_INIT = new Object();
+
+  MergeUnionEnumerator(
+      List<Enumerable<TSource>> sources,
+      Function1<TSource, TKey> sortKeySelector,
+      Comparator<TKey> sortComparator,
+      boolean all,
+      EqualityComparer<TSource> equalityComparer) {
+    this.sortKeySelector = sortKeySelector;
+    this.sortComparator = sortComparator;
+
+    if (all) {
+      this.processed = null;
+      this.wrapper = null;
+    } else {
+      this.processed = new HashSet<>();
+      this.wrapper = EnumerableDefaults.wrapperFor(equalityComparer);
+    }
+
+    final int size = sources.size();
+    //noinspection unchecked
+    this.inputs = new Enumerator[size];
+    int i = 0;
+    for (Enumerable<TSource> source : sources) {
+      this.inputs[i++] = source.enumerator();
+    }
+
+    //noinspection unchecked
+    this.currentInputsValues = (TSource[]) new Object[size];
+    this.activeInputs = this.currentInputsValues.length;
+    this.inputsFinished = new boolean[size];
+    //noinspection unchecked
+    this.currentValue = (TSource) NOT_INIT;
+
+    initEnumerators();
+  }
+
+  @RequiresNonNull("inputs")
+  @SuppressWarnings("method.invocation.invalid")
+  private void initEnumerators(@UnknownInitialization MergeUnionEnumerator<TSource, TKey> this) {
+    for (int i = 0; i < inputs.length; i++) {
+      moveEnumerator(i);
+    }
+  }
+
+  private void moveEnumerator(int i) {
+    final Enumerator<TSource> enumerator = inputs[i];
+    if (!enumerator.moveNext()) {
+      activeInputs--;
+      inputsFinished[i] = true;

Review comment:
       Thanks for the feedback, I applied your first WA.




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

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



[GitHub] [calcite] rubenada commented on a change in pull request #2329: [CALCITE-3221] Add a sort-merge union algorithm

Posted by GitBox <gi...@apache.org>.
rubenada commented on a change in pull request #2329:
URL: https://github.com/apache/calcite/pull/2329#discussion_r562018414



##########
File path: linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
##########
@@ -4683,4 +4683,191 @@ private void flush() {
       }
     };
   }
+
+  /**
+   * Merge Union Enumerable.
+   * Performs a union (or union all) of all its inputs (which must be already sorted),
+   * respecting the order.
+   *
+   * @param sources input enumerables (must be already sorted)
+   * @param sortKeySelector sort key selector
+   * @param sortComparator sort comparator to decide the next item
+   * @param all whether duplicates will be considered or not
+   * @param equalityComparer {@link EqualityComparer} to control duplicates,
+   *                         only used if {@code all} is {@code false}
+   * @param <TSource> record type
+   * @param <TKey> sort key
+   */
+  public static <TSource, @Nullable TKey> Enumerable<TSource> mergeUnion(

Review comment:
       You're right, in other parts of `EnumerableDefaults` using `Function1<TSource, TKey> keySelector` there is no null check before calling `keySelector.apply`, I'll remove the null handling here.




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

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