You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tl...@apache.org on 2021/06/23 08:29:58 UTC
[ignite-3] branch main updated: IGNITE-14835 Calcite SQL engine.
Move execution related code to Ignite 3.0 (#182)
This is an automated email from the ASF dual-hosted git repository.
tledkov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new 972672f IGNITE-14835 Calcite SQL engine. Move execution related code to Ignite 3.0 (#182)
972672f is described below
commit 972672ffcd7b0b4bdc31b3511fdcb90f51cc99ec
Author: korlov42 <ko...@gridgain.com>
AuthorDate: Wed Jun 23 11:29:49 2021 +0300
IGNITE-14835 Calcite SQL engine. Move execution related code to Ignite 3.0 (#182)
---
.../internal/processors/query/calcite/Stubs.java | 6 -
.../query/calcite/exec/AbstractIndexScan.java | 107 +
.../query/calcite/exec/ArrayRowHandler.java | 77 +
.../ExecutionCancelledException.java} | 24 +-
.../query/calcite/exec/ExecutionContext.java | 292 +++
.../query/calcite/exec/PlannerHelper.java | 229 ++
.../query/calcite/exec/QueryTaskExecutor.java | 47 +
.../query/calcite/exec/QueryTaskExecutorImpl.java | 95 +
.../processors/query/calcite/exec/RowHandler.java | 73 +
.../query/calcite/exec/RuntimeHashIndex.java | 113 +
.../calcite/{Stubs.java => exec/RuntimeIndex.java} | 32 +-
.../query/calcite/exec/RuntimeTreeIndex.java | 196 ++
.../query/calcite/exec/TableFunctionScan.java | 48 +
.../calcite/{Stubs.java => exec/TreeIndex.java} | 37 +-
.../query/calcite/exec/exp/CallImplementor.java | 42 +
.../query/calcite/exec/exp/ConverterUtils.java | 460 ++++
.../query/calcite/exec/exp/ExpressionFactory.java | 125 +
.../calcite/exec/exp/ExpressionFactoryImpl.java | 509 ++++
.../calcite/exec/exp/IgniteBuiltInMethod.java | 45 +
.../query/calcite/exec/exp/IgniteSqlFunctions.java | 154 ++
.../calcite/exec/exp/ImplementableFunction.java | 36 +
.../query/calcite/exec/exp/NotNullImplementor.java | 43 +
.../exec/exp/ReflectiveCallNotNullImplementor.java | 82 +
.../query/calcite/exec/exp/RexImpTable.java | 2574 ++++++++++++++++++++
.../query/calcite/exec/exp/RexToLixTranslator.java | 1312 ++++++++++
.../calcite/{Stubs.java => exec/exp/Scalar.java} | 24 +-
.../{Stubs.java => exec/exp/agg/Accumulator.java} | 32 +-
.../exp/agg/AccumulatorWrapper.java} | 23 +-
.../query/calcite/exec/exp/agg/Accumulators.java | 1068 ++++++++
.../calcite/exec/exp/agg/AccumulatorsFactory.java | 330 +++
.../exp/agg/AggregateType.java} | 30 +-
.../query/calcite/exec/exp/agg/GroupKey.java | 101 +
.../query/calcite/exec/rel/AbstractNode.java | 202 ++
.../query/calcite/exec/rel/AbstractSetOpNode.java | 367 +++
.../exec/rel/CorrelatedNestedLoopJoinNode.java | 479 ++++
.../{Stubs.java => exec/rel/Downstream.java} | 43 +-
.../query/calcite/exec/rel/FilterNode.java | 145 ++
.../query/calcite/exec/rel/HashAggregateNode.java | 372 +++
.../query/calcite/exec/rel/IndexSpoolNode.java | 205 ++
.../query/calcite/exec/rel/IntersectNode.java | 118 +
.../query/calcite/exec/rel/LimitNode.java | 132 +
.../calcite/{Stubs.java => exec/rel/Mailbox.java} | 36 +-
.../query/calcite/exec/rel/MergeJoinNode.java | 1134 +++++++++
.../query/calcite/exec/rel/MinusNode.java | 102 +
.../query/calcite/exec/rel/NestedLoopJoinNode.java | 812 ++++++
.../processors/query/calcite/exec/rel/Node.java | 81 +
.../query/calcite/exec/rel/ProjectNode.java | 84 +
.../query/calcite/exec/rel/RootNode.java | 283 +++
.../query/calcite/exec/rel/ScanNode.java | 130 +
.../{Stubs.java => exec/rel/SingleNode.java} | 29 +-
.../query/calcite/exec/rel/SortAggregateNode.java | 316 +++
.../query/calcite/exec/rel/SortNode.java | 148 ++
.../query/calcite/exec/rel/TableSpoolNode.java | 160 ++
.../query/calcite/exec/rel/UnionAllNode.java | 97 +
.../calcite/metadata/FragmentDescription.java | 75 +
.../calcite/metadata/cost/IgniteCostFactory.java | 4 +-
.../calcite/prepare/ddl/ColumnDefinition.java | 2 +-
.../processors/query/calcite/util/Commons.java | 27 +-
.../query/calcite/util/FilteringIterator.java | 79 +
.../query/calcite/util/IgniteMethod.java | 25 +
.../processors/query/calcite/util/RexUtils.java | 8 +-
.../calcite/util/StripedThreadPoolExecutor.java | 226 ++
.../{Stubs.java => util/TransformingIterator.java} | 37 +-
.../processors/query/calcite/util/TypeUtils.java | 111 +
.../query/calcite/exec/RuntimeTreeIndexTest.java | 199 ++
.../calcite/exec/rel/AbstractExecutionTest.java | 226 ++
.../exec/rel/AbstractSetOpExecutionTest.java | 166 ++
.../query/calcite/exec/rel/BaseAggregateTest.java | 552 +++++
.../query/calcite/exec/rel/ExecutionTest.java | 667 +++++
.../exec/rel/HashAggregateExecutionTest.java | 135 +
.../rel/HashAggregateSingleGroupExecutionTest.java | 497 ++++
.../exec/rel/HashIndexSpoolExecutionTest.java | 167 ++
.../calcite/exec/rel/IntersectExecutionTest.java | 83 +
.../calcite/exec/rel/MergeJoinExecutionTest.java | 378 +++
.../query/calcite/exec/rel/MinusExecutionTest.java | 83 +
.../exec/rel/NestedLoopJoinExecutionTest.java | 363 +++
.../exec/rel/SortAggregateExecutionTest.java | 141 ++
.../calcite/exec/rel/TableSpoolExecutionTest.java | 152 ++
.../exec/rel/TreeIndexSpoolExecutionTest.java | 194 ++
.../apache/ignite/internal/util/ArrayUtils.java | 2 +
.../apache/ignite/internal/util/IgniteUtils.java | 30 +-
.../internal/testframework/IgniteTestUtils.java | 71 +
.../internal/util/IgniteIntListSelfTest.java | 159 ++
83 files changed, 18485 insertions(+), 215 deletions(-)
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
index ce7d926..093146c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
@@ -23,12 +23,6 @@ public class Stubs {
return args == null ? 0 : args.length;
}
- /** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
-
- /** */
public static boolean boolFoo(Object... args) {
return args == null;
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java
new file mode 100644
index 0000000..fd61d52
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.util.Iterator;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.util.FilteringIterator;
+import org.apache.ignite.internal.processors.query.calcite.util.TransformingIterator;
+import org.apache.ignite.lang.IgniteInternalException;
+
+/**
+ * Abstract index scan.
+ */
+public abstract class AbstractIndexScan<Row, IdxRow> implements Iterable<Row>, AutoCloseable {
+ /** */
+ private final TreeIndex<IdxRow> idx;
+
+ /** Additional filters. */
+ private final Predicate<Row> filters;
+
+ /** Lower index scan bound. */
+ private final Supplier<Row> lowerBound;
+
+ /** Upper index scan bound. */
+ private final Supplier<Row> upperBound;
+
+ /** */
+ private final Function<Row, Row> rowTransformer;
+
+ /** */
+ protected final ExecutionContext<Row> ectx;
+
+ /** */
+ protected final RelDataType rowType;
+
+ /**
+ * @param ectx Execution context.
+ * @param idx Physical index.
+ * @param filters Additional filters.
+ * @param lowerBound Lower index scan bound.
+ * @param upperBound Upper index scan bound.
+ */
+ protected AbstractIndexScan(
+ ExecutionContext<Row> ectx,
+ RelDataType rowType,
+ TreeIndex<IdxRow> idx,
+ Predicate<Row> filters,
+ Supplier<Row> lowerBound,
+ Supplier<Row> upperBound,
+ Function<Row, Row> rowTransformer
+ ) {
+ this.ectx = ectx;
+ this.rowType = rowType;
+ this.idx = idx;
+ this.filters = filters;
+ this.lowerBound = lowerBound;
+ this.upperBound = upperBound;
+ this.rowTransformer = rowTransformer;
+ }
+
+ /** {@inheritDoc} */
+ @Override public synchronized Iterator<Row> iterator() {
+ IdxRow lower = lowerBound == null ? null : row2indexRow(lowerBound.get());
+ IdxRow upper = upperBound == null ? null : row2indexRow(upperBound.get());
+
+ Iterator<Row> it = new TransformingIterator<>(
+ idx.find(lower, upper),
+ this::indexRow2Row
+ );
+
+ it = new FilteringIterator<>(it, filters);
+
+ if (rowTransformer != null)
+ it = new TransformingIterator<>(it, rowTransformer);
+
+ return it;
+ }
+
+ /** */
+ protected abstract IdxRow row2indexRow(Row bound);
+
+ /** */
+ protected abstract Row indexRow2Row(IdxRow idxRow) throws IgniteInternalException;
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ // No-op.
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java
new file mode 100644
index 0000000..f6b9274
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ArrayRowHandler.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.lang.reflect.Type;
+
+import org.apache.ignite.internal.util.ArrayUtils;
+
+/**
+ * Handler for rows that implemented as a simple objects array.
+ */
+public class ArrayRowHandler implements RowHandler<Object[]> {
+ /** */
+ public static final RowHandler<Object[]> INSTANCE = new ArrayRowHandler();
+
+ /** */
+ private ArrayRowHandler() {}
+
+ /** {@inheritDoc} */
+ @Override public Object get(int field, Object[] row) {
+ return row[field];
+ }
+
+ /** {@inheritDoc} */
+ @Override public void set(int field, Object[] row, Object val) {
+ row[field] = val;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object[] concat(Object[] left, Object[] right) {
+ return ArrayUtils.concat(left, right);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int columnCount(Object[] row) {
+ return row.length;
+ }
+
+ /** {@inheritDoc} */
+ @Override public RowFactory<Object[]> factory(Type... types) {
+ int rowLen = types.length;
+
+ return new RowFactory<>() {
+ /** {@inheritDoc} */
+ @Override public RowHandler<Object[]> handler() {
+ return ArrayRowHandler.this;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object[] create() {
+ return new Object[rowLen];
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object[] create(Object... fields) {
+ assert fields.length == rowLen;
+
+ return fields;
+ }
+ };
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
index ce7d926..ce2e4c7 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionCancelledException.java
@@ -14,27 +14,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite;
-/** Stubs */
-public class Stubs {
- /** */
- public static int intFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+package org.apache.ignite.internal.processors.query.calcite.exec;
- /** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+import org.apache.ignite.lang.IgniteInternalCheckedException;
- /** */
- public static boolean boolFoo(Object... args) {
- return args == null;
- }
-
- /** */
- public static String stringFoo(Object... args) {
- return args == null ? "null" : "not null";
- }
+/** */
+public class ExecutionCancelledException extends IgniteInternalCheckedException {
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
new file mode 100644
index 0000000..96ad599
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactoryImpl;
+import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
+import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.Commons.checkRange;
+
+/**
+ * Runtime context allowing access to the tables in a database.
+ */
+public class ExecutionContext<Row> implements DataContext {
+ /** */
+ private static final TimeZone TIME_ZONE = TimeZone.getDefault(); // TODO DistributedSqlConfiguration#timeZone
+
+ /** */
+ private final UUID qryId;
+
+ /** */
+ private final PlanningContext ctx;
+
+ /** */
+ private final FragmentDescription fragmentDesc;
+
+ /** */
+ private final Map<String, Object> params;
+
+ /** */
+ private final QueryTaskExecutor executor;
+
+ /** */
+ private final RowHandler<Row> handler;
+
+ /** */
+ private final ExpressionFactory<Row> expressionFactory;
+
+ /** */
+ private final AtomicBoolean cancelFlag = new AtomicBoolean();
+
+ /** */
+ private Object[] correlations = new Object[16];
+
+ /**
+ * @param executor Task executor.
+ * @param ctx Parent context.
+ * @param qryId Query ID.
+ * @param fragmentDesc Partitions information.
+ * @param handler Row handler.
+ * @param params Parameters.
+ */
+ @SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+ public ExecutionContext(
+ QueryTaskExecutor executor,
+ PlanningContext ctx,
+ UUID qryId,
+ FragmentDescription fragmentDesc,
+ RowHandler<Row> handler,
+ Map<String, Object> params
+ ) {
+ this.executor = executor;
+ this.ctx = ctx;
+ this.qryId = qryId;
+ this.fragmentDesc = fragmentDesc;
+ this.handler = handler;
+ this.params = params;
+
+ expressionFactory = new ExpressionFactoryImpl<>(this, ctx.typeFactory(), ctx.conformance());
+ }
+
+ /**
+ * @return Parent context.
+ */
+ public PlanningContext planningContext() {
+ return ctx;
+ }
+
+ /**
+ * @return Query ID.
+ */
+ public UUID queryId() {
+ return qryId;
+ }
+
+ /**
+ * @return Fragment ID.
+ */
+ public long fragmentId() {
+ return fragmentDesc.fragmentId();
+ }
+
+ /**
+ * @return Target mapping.
+ */
+ public ColocationGroup target() {
+ return fragmentDesc.target();
+ }
+
+ /**
+ * @param exchangeId ExchangeId to find remote nodes for.
+ * @return Remote nodes for given exchangeId.
+ */
+ public List<UUID> remotes(long exchangeId) {
+ return fragmentDesc.remotes().get(exchangeId);
+ }
+
+ /**
+ * @param sourceId SourceId to find colocation group for.
+ * @return Colocation group for given sourceId.
+ */
+ public ColocationGroup group(long sourceId) {
+ return fragmentDesc.mapping().findGroup(sourceId);
+ }
+
+ /**
+ * @return Keep binary flag.
+ */
+ public boolean keepBinary() {
+ return true; // TODO
+ }
+
+ /**
+ * @return Handler to access row fields.
+ */
+ public RowHandler<Row> rowHandler() {
+ return handler;
+ }
+
+ /**
+ * @return Expression factory.
+ */
+ public ExpressionFactory<Row> expressionFactory() {
+ return expressionFactory;
+ }
+
+ /**
+ * @return Originating node ID.
+ */
+ public UUID originatingNodeId() {
+ return planningContext().originatingNodeId();
+ }
+
+ /** {@inheritDoc} */
+ @Override public SchemaPlus getRootSchema() {
+ return ctx.schema();
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteTypeFactory getTypeFactory() {
+ return ctx.typeFactory();
+ }
+
+ /** {@inheritDoc} */
+ @Override public QueryProvider getQueryProvider() {
+ return null; // TODO
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object get(String name) {
+ if (Variable.CANCEL_FLAG.camelName.equals(name))
+ return cancelFlag;
+ if (Variable.TIME_ZONE.camelName.equals(name))
+ return TIME_ZONE; // TODO DistributedSqlConfiguration#timeZone
+ if (name.startsWith("?"))
+ return TypeUtils.toInternal(this, params.get(name));
+
+ return params.get(name);
+ }
+
+ /**
+ * Gets correlated value.
+ *
+ * @param id Correlation ID.
+ * @return Correlated value.
+ */
+ public @NotNull Object getCorrelated(int id) {
+ checkRange(correlations, id);
+
+ return correlations[id];
+ }
+
+ /**
+ * Sets correlated value.
+ *
+ * @param id Correlation ID.
+ * @param value Correlated value.
+ */
+ public void setCorrelated(@NotNull Object value, int id) {
+ correlations = Commons.ensureCapacity(correlations, id + 1);
+
+ correlations[id] = value;
+ }
+
+ /**
+ * Executes a query task.
+ *
+ * @param task Query task.
+ */
+ public void execute(RunnableX task, Consumer<Throwable> onError) {
+ if (isCancelled())
+ return;
+
+ executor.execute(qryId, fragmentId(), () -> {
+ try {
+ task.run();
+ }
+ catch (Throwable e) {
+ onError.accept(e);
+
+ throw new IgniteInternalException("Unexpected exception", e);
+ }
+ });
+ }
+
+ /**
+ * Submits a Runnable task for execution and returns a Future
+ * representing that task. The Future's {@code get} method will
+ * return {@code null} upon <em>successful</em> completion.
+ *
+ * @param task the task to submit.
+ * @return a {@link CompletableFuture} representing pending task
+ */
+ public CompletableFuture<?> submit(RunnableX task, Consumer<Throwable> onError) {
+ assert !isCancelled() : "Call submit after execution was cancelled.";
+
+ return executor.submit(qryId, fragmentId(), () -> {
+ try {
+ task.run();
+ }
+ catch (Throwable e) {
+ onError.accept(e);
+
+ throw new IgniteInternalException("Unexpected exception", e);
+ }
+ });
+ }
+
+ /** */
+ @FunctionalInterface
+ public interface RunnableX {
+ /** */
+ void run() throws Exception;
+ }
+
+ /**
+ * Sets cancel flag, returns {@code true} if flag was changed by this call.
+ *
+ * @return {@code True} if flag was changed by this call.
+ */
+ public boolean cancel() {
+ return !cancelFlag.get() && cancelFlag.compareAndSet(false, true);
+ }
+
+ /** */
+ public boolean isCancelled() {
+ return cancelFlag.get();
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PlannerHelper.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PlannerHelper.java
new file mode 100644
index 0000000..4d53326
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/PlannerHelper.java
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.core.Spool;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgnitePlanner;
+import org.apache.ignite.internal.processors.query.calcite.prepare.IgniteRelShuttle;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlannerPhase;
+import org.apache.ignite.internal.processors.query.calcite.rel.AbstractIndexScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteConvention;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteIndexScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteProject;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableScan;
+import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableSpool;
+import org.apache.ignite.internal.processors.query.calcite.schema.ColumnDescriptor;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
+import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.HintUtils;
+import org.apache.ignite.lang.IgniteLogger;
+
+/** */
+public class PlannerHelper {
+ /**
+ * Default constructor.
+ */
+ private PlannerHelper() {
+
+ }
+
+ /**
+ * @param sqlNode Sql node.
+ * @param planner Planner.
+ * @param log Logger.
+ */
+ public static IgniteRel optimize(SqlNode sqlNode, IgnitePlanner planner, IgniteLogger log) {
+ try {
+ // Convert to Relational operators graph
+ RelRoot root = planner.rel(sqlNode);
+
+ RelNode rel = root.rel;
+
+ if (HintUtils.containsDisabledRules(root.hints))
+ planner.setDisabledRules(HintUtils.disabledRules(root.hints));
+
+ // Transformation chain
+ rel = planner.transform(PlannerPhase.HEURISTIC_OPTIMIZATION, rel.getTraitSet(), rel);
+
+ RelTraitSet desired = rel.getCluster().traitSet()
+ .replace(IgniteConvention.INSTANCE)
+ .replace(IgniteDistributions.single())
+ .replace(root.collation == null ? RelCollations.EMPTY : root.collation)
+ .simplify();
+
+ IgniteRel igniteRel = planner.transform(PlannerPhase.OPTIMIZATION, desired, rel);
+
+ if (!root.isRefTrivial()) {
+ final List<RexNode> projects = new ArrayList<>();
+ final RexBuilder rexBuilder = igniteRel.getCluster().getRexBuilder();
+
+ for (int field : Pair.left(root.fields))
+ projects.add(rexBuilder.makeInputRef(igniteRel, field));
+
+ igniteRel = new IgniteProject(igniteRel.getCluster(), desired, igniteRel, projects, root.validatedRowType);
+ }
+
+ if (sqlNode.isA(ImmutableSet.of(SqlKind.INSERT, SqlKind.UPDATE, SqlKind.MERGE)))
+ igniteRel = new FixDependentModifyNodeShuttle().visit(igniteRel);
+
+ return igniteRel;
+ }
+ catch (Throwable ex) {
+ log.error("Unexpected error at query optimizer.", ex);
+ log.error(planner.dump());
+
+ throw ex;
+ }
+ }
+
+ /**
+ * This shuttle analyzes a relational tree and inserts an eager spool node
+ * just under the TableModify node in case latter depends upon a table used
+ * to query the data for modify node to avoid the double processing
+ * of the retrieved rows.
+ * <p/>
+ * It considers two cases: <ol>
+ * <li>
+ * Modify node produces rows to insert, then a spool is required.
+ * </li>
+ * <li>
+ * Modify node updates rows only, then a spool is required if 1) we
+ * are scaning an index and 2) any of the indexed column is updated
+ * by modify node.
+ * </li>
+ * <ol/>
+ *
+ */
+ private static class FixDependentModifyNodeShuttle extends IgniteRelShuttle {
+ /**
+ * Flag indicates whether we should insert a spool or not.
+ */
+ private boolean spoolNeeded;
+
+ /** Current modify node. */
+ private IgniteTableModify modifyNode;
+
+ /** {@inheritDoc} */
+ @Override public IgniteRel visit(IgniteTableModify rel) {
+ assert modifyNode == null;
+
+ modifyNode = rel;
+
+ if (rel.isDelete())
+ return rel;
+
+ processNode(rel);
+
+ if (spoolNeeded) {
+ IgniteTableSpool spool = new IgniteTableSpool(
+ rel.getCluster(),
+ rel.getInput().getTraitSet(),
+ Spool.Type.EAGER,
+ rel.getInput()
+ );
+
+ rel.replaceInput(0, spool);
+ }
+
+ return rel;
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteRel visit(IgniteTableScan rel) {
+ return processScan(rel);
+ }
+
+ /** {@inheritDoc} */
+ @Override public IgniteRel visit(IgniteIndexScan rel) {
+ return processScan(rel);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected IgniteRel processNode(IgniteRel rel) {
+ List<IgniteRel> inputs = Commons.cast(rel.getInputs());
+
+ for (int i = 0; i < inputs.size(); i++) {
+ if (spoolNeeded)
+ break;
+
+ visitChild(rel, i, inputs.get(i));
+ }
+
+ return rel;
+ }
+
+ /**
+ * Process a scan node and raise a {@link #spoolNeeded flag} if needed.
+ *
+ * @param scan TableScan to analize.
+ * @return The input rel.
+ */
+ private IgniteRel processScan(TableScan scan) {
+ IgniteTable tbl = modifyNode != null ? modifyNode.getTable().unwrap(IgniteTable.class) : null;
+
+ if (tbl == null || scan.getTable().unwrap(IgniteTable.class) != tbl)
+ return (IgniteRel)scan;
+
+ if (modifyNodeInsertsData()) {
+ spoolNeeded = true;
+
+ return (IgniteRel)scan;
+ }
+
+ // for update-only node the spool needed if any of the updated
+ // column is part of the index we are going to scan
+ if (scan instanceof IgniteTableScan)
+ return (IgniteRel)scan;
+
+ ImmutableSet<Integer> indexedCols = ImmutableSet.copyOf(
+ tbl.getIndex(((AbstractIndexScan)scan).indexName()).collation().getKeys());
+
+ spoolNeeded = modifyNode.getUpdateColumnList().stream()
+ .map(tbl.descriptor()::columnDescriptor)
+ .map(ColumnDescriptor::fieldIndex)
+ .anyMatch(indexedCols::contains);
+
+ return (IgniteRel)scan;
+ }
+
+ /**
+ * @return {@code true} in case {@link #modifyNode} produces any insert.
+ */
+ private boolean modifyNodeInsertsData() {
+ return modifyNode.isInsert(); // MERGE should be analyzed too
+ // but currently it is not implemented
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java
new file mode 100644
index 0000000..b4a5fb2
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ *
+ */
+public interface QueryTaskExecutor {
+ /**
+ * Executes a query task in a thread, responsible for particular query fragment.
+ *
+ * @param qryId Query ID.
+ * @param fragmentId Fragment ID.
+ * @param qryTask Query task.
+ */
+ void execute(UUID qryId, long fragmentId, Runnable qryTask);
+
+ /**
+ * Returns a new CompletableFuture that is asynchronously completed
+ * by a task running in the given executor after it runs the given
+ * action.
+ *
+ * @param qryId Id of the query this task created for.
+ * @param fragmentId Id of the particular fragment this task created for.
+ * @param qryTask The task to submit.
+ * @return the new CompletableFuture
+ */
+ CompletableFuture<?> submit(UUID qryId, long fragmentId, Runnable qryTask);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
new file mode 100644
index 0000000..abcad0a
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/QueryTaskExecutorImpl.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.ignite.internal.processors.query.calcite.util.StripedThreadPoolExecutor;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteLogger;
+
+/** */
+public class QueryTaskExecutorImpl implements QueryTaskExecutor, Thread.UncaughtExceptionHandler {
+ /** */
+ private static final IgniteLogger LOG = IgniteLogger.forClass(QueryTaskExecutorImpl.class);
+
+ /** */
+ private StripedThreadPoolExecutor stripedThreadPoolExecutor;
+
+ /** */
+ private Thread.UncaughtExceptionHandler eHnd;
+
+ /**
+ * @param stripedThreadPoolExecutor Executor.
+ */
+ public void stripedThreadPoolExecutor(StripedThreadPoolExecutor stripedThreadPoolExecutor) {
+ this.stripedThreadPoolExecutor = stripedThreadPoolExecutor;
+ }
+
+ /**
+ * @param eHnd Uncaught exception handler.
+ */
+ public void exceptionHandler(Thread.UncaughtExceptionHandler eHnd) {
+ this.eHnd = eHnd;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute(UUID qryId, long fragmentId, Runnable qryTask) {
+ stripedThreadPoolExecutor.execute(
+ () -> {
+ try {
+ qryTask.run();
+ }
+ catch (Throwable e) {
+ LOG.warn("Uncaught exception", e);
+
+ /*
+ * No exceptions are rethrown here to preserve the current thread from being destroyed,
+ * because other queries may be pinned to the current thread id.
+ * However, unrecoverable errors must be processed by FailureHandler.
+ */
+ uncaughtException(Thread.currentThread(), e);
+ }
+ },
+ hash(qryId, fragmentId)
+ );
+ }
+
+ /** {@inheritDoc} */
+ @Override public CompletableFuture<?> submit(UUID qryId, long fragmentId, Runnable qryTask) {
+ return stripedThreadPoolExecutor.submit(qryTask, hash(qryId, fragmentId));
+ }
+
+ /** Releases resources. */
+ public void tearDown() {
+ stripedThreadPoolExecutor.shutdownNow();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void uncaughtException(Thread t, Throwable e) {
+ if (eHnd != null)
+ eHnd.uncaughtException(t, e);
+ }
+
+ /** */
+ private static int hash(UUID qryId, long fragmentId) {
+ // inlined Objects.hash(...)
+ return IgniteUtils.safeAbs(31 * (31 + (qryId != null ? qryId.hashCode() : 0)) + Long.hashCode(fragmentId));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java
new file mode 100644
index 0000000..6dac05c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RowHandler.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.lang.reflect.Type;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/**
+ * Universal accessor and mutator for rows. It also has factory methods.
+ */
+public interface RowHandler<Row> {
+ /** */
+ Object get(int field, Row row);
+
+ /** */
+ void set(int field, Row row, Object val);
+
+ /** */
+ Row concat(Row left, Row right);
+
+ /** */
+ int columnCount(Row row);
+
+ /** */
+ default RowFactory<Row> factory(IgniteTypeFactory typeFactory, RelDataType rowType) {
+ if (rowType.isStruct())
+ return factory(typeFactory, RelOptUtil.getFieldTypeList(rowType));
+
+ return factory(typeFactory.getJavaClass(rowType));
+ }
+
+ /** */
+ default RowFactory<Row> factory(IgniteTypeFactory typeFactory, List<RelDataType> fieldTypes) {
+ Type[] types = new Type[fieldTypes.size()];
+ for (int i = 0; i < fieldTypes.size(); i++)
+ types[i] = typeFactory.getJavaClass(fieldTypes.get(i));
+
+ return factory(types);
+ }
+
+ RowFactory<Row> factory(Type... types);
+
+ @SuppressWarnings("PublicInnerClass")
+ interface RowFactory<Row> {
+ /** */
+ RowHandler<Row> handler();
+
+ /** */
+ Row create();
+
+ /** */
+ Row create(Object... fields);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java
new file mode 100644
index 0000000..03abac6
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeHashIndex.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Supplier;
+
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * Runtime hash index based on on-heap hash map.
+ */
+public class RuntimeHashIndex<Row> implements RuntimeIndex<Row> {
+ /** */
+ protected final ExecutionContext<Row> ectx;
+
+ /** */
+ private final ImmutableBitSet keys;
+
+ /** Rows. */
+ private HashMap<GroupKey, List<Row>> rows;
+
+ /**
+ *
+ */
+ public RuntimeHashIndex(
+ ExecutionContext<Row> ectx,
+ ImmutableBitSet keys
+ ) {
+ this.ectx = ectx;
+
+ assert !nullOrEmpty(keys);
+
+ this.keys = keys;
+ rows = new HashMap<>();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void push(Row r) {
+ List<Row> eqRows = rows.computeIfAbsent(key(r), k -> new ArrayList<>());
+
+ eqRows.add(r);
+ }
+
+ /** */
+ @Override public void close() {
+ rows.clear();
+ }
+
+ /** */
+ public Iterable<Row> scan(Supplier<Row> searchRow) {
+ return new IndexScan(searchRow);
+ }
+
+ /** */
+ private GroupKey key(Row r) {
+ GroupKey.Builder b = GroupKey.builder(keys.cardinality());
+
+ for (Integer field : keys)
+ b.add(ectx.rowHandler().get(field, r));
+
+ return b.build();
+ }
+
+ /**
+ *
+ */
+ private class IndexScan implements Iterable<Row>, AutoCloseable {
+ /** Search row. */
+ private final Supplier<Row> searchRow;
+
+ /**
+ * @param searchRow Search row.
+ */
+ IndexScan(Supplier<Row> searchRow) {
+ this.searchRow = searchRow;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ // No-op.
+ }
+
+ /** {@inheritDoc} */
+ @NotNull @Override public Iterator<Row> iterator() {
+ List<Row> eqRows = rows.get(key(searchRow.get()));
+
+ return eqRows == null ? Collections.emptyIterator() : eqRows.iterator();
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeIndex.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeIndex.java
index ce7d926..01d257c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeIndex.java
@@ -14,27 +14,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite;
+package org.apache.ignite.internal.processors.query.calcite.exec;
-/** Stubs */
-public class Stubs {
- /** */
- public static int intFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
-
- /** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
-
- /** */
- public static boolean boolFoo(Object... args) {
- return args == null;
- }
-
- /** */
- public static String stringFoo(Object... args) {
- return args == null ? "null" : "not null";
- }
+/**
+ * Runtime index interface.
+ * The temporary index is built and available only on query execution. Not stored at the schema.
+ */
+public interface RuntimeIndex<Row> extends AutoCloseable {
+ /**
+ * Add row to index.
+ */
+ void push(Row r);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
new file mode 100644
index 0000000..474f5dc
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeTreeIndex.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+
+/**
+ * Runtime sorted index based on on-heap tree.
+ */
+public class RuntimeTreeIndex<Row> implements RuntimeIndex<Row>, TreeIndex<Row> {
+ /** */
+ protected final ExecutionContext<Row> ectx;
+
+ /** */
+ protected final Comparator<Row> comp;
+
+ /** Collation. */
+ private final RelCollation collation;
+
+ /** Rows. */
+ private TreeMap<Row, List<Row>> rows;
+
+ /**
+ *
+ */
+ public RuntimeTreeIndex(
+ ExecutionContext<Row> ectx,
+ RelCollation collation,
+ Comparator<Row> comp
+ ) {
+ this.ectx = ectx;
+ this.comp = comp;
+
+ assert Objects.nonNull(collation);
+
+ this.collation = collation;
+ rows = new TreeMap<>(comp);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void push(Row r) {
+ List<Row> newEqRows = new ArrayList<>();
+
+ List<Row> eqRows = rows.putIfAbsent(r, newEqRows);
+
+ if (eqRows != null)
+ eqRows.add(r);
+ else
+ newEqRows.add(r);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ rows.clear();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Cursor<Row> find(Row lower, Row upper) {
+ int firstCol = first(collation.getKeys());
+
+ if (ectx.rowHandler().get(firstCol, lower) != null && ectx.rowHandler().get(firstCol, upper) != null)
+ return new CursorImpl(rows.subMap(lower, true, upper, true));
+ else if (ectx.rowHandler().get(firstCol, lower) == null && ectx.rowHandler().get(firstCol, upper) != null)
+ return new CursorImpl(rows.headMap(upper, true));
+ else if (ectx.rowHandler().get(firstCol, lower) != null && ectx.rowHandler().get(firstCol, upper) == null)
+ return new CursorImpl(rows.tailMap(lower, true));
+ else
+ return new CursorImpl(rows);
+ }
+
+ /**
+ * Creates iterable on the index.
+ */
+ public Iterable<Row> scan(
+ ExecutionContext<Row> ectx,
+ RelDataType rowType,
+ Predicate<Row> filter,
+ Supplier<Row> lowerBound,
+ Supplier<Row> upperBound
+ ) {
+ return new IndexScan(rowType, this, filter, lowerBound, upperBound);
+ }
+
+ /**
+ *
+ */
+ private class CursorImpl implements Cursor<Row> {
+ /** Sub map iterator. */
+ private final Iterator<Map.Entry<Row, List<Row>>> mapIt;
+
+ /** Iterator over rows with equal index keys. */
+ private Iterator<Row> listIt;
+
+ /** */
+ private Row row;
+
+ /** */
+ CursorImpl(SortedMap<Row, List<Row>> subMap) {
+ mapIt = subMap.entrySet().iterator();
+ listIt = null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Row next() throws IgniteInternalException {
+ if (!hasNext())
+ throw new NoSuchElementException();
+
+ advance();
+
+ return listIt.next();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean hasNext() {
+ return listIt != null && listIt.hasNext() || mapIt.hasNext();
+ }
+
+ /** */
+ private void advance() {
+ if (listIt == null || !listIt.hasNext())
+ listIt = mapIt.next().getValue().iterator();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() throws Exception {
+ }
+
+ /** {@inheritDoc} */
+ @NotNull @Override public Iterator<Row> iterator() {
+ return this;
+ }
+ }
+
+ /**
+ *
+ */
+ private class IndexScan extends AbstractIndexScan<Row, Row> {
+ /**
+ * @param rowType Row type.
+ * @param idx Physical index.
+ * @param filter Additional filters.
+ * @param lowerBound Lower index scan bound.
+ * @param upperBound Upper index scan bound.
+ */
+ IndexScan(
+ RelDataType rowType,
+ TreeIndex<Row> idx,
+ Predicate<Row> filter,
+ Supplier<Row> lowerBound,
+ Supplier<Row> upperBound) {
+ super(RuntimeTreeIndex.this.ectx, rowType, idx, filter, lowerBound, upperBound, null);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Row row2indexRow(Row bound) {
+ return bound;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Row indexRow2Row(Row row) {
+ return row;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableFunctionScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableFunctionScan.java
new file mode 100644
index 0000000..9744a04
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableFunctionScan.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec;
+
+import java.util.Iterator;
+import java.util.function.Supplier;
+
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.TransformingIterator;
+import org.jetbrains.annotations.NotNull;
+
+/** */
+public class TableFunctionScan<Row> implements Iterable<Row> {
+ /** */
+ private final Supplier<Iterable<Object[]>> dataSupplier;
+
+ /** */
+ private final RowFactory<Row> rowFactory;
+
+ /** */
+ public TableFunctionScan(
+ Supplier<Iterable<Object[]>> dataSupplier,
+ RowFactory<Row> rowFactory
+ ) {
+ this.dataSupplier = dataSupplier;
+ this.rowFactory = rowFactory;
+ }
+
+ /** {@inheritDoc} */
+ @NotNull @Override public Iterator<Row> iterator() {
+ return new TransformingIterator<>(dataSupplier.get().iterator(), rowFactory::create);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TreeIndex.java
similarity index 59%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TreeIndex.java
index ce7d926..9911df0 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TreeIndex.java
@@ -14,27 +14,22 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite;
+package org.apache.ignite.internal.processors.query.calcite.exec;
-/** Stubs */
-public class Stubs {
- /** */
- public static int intFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+import org.apache.ignite.internal.util.Cursor;
- /** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
-
- /** */
- public static boolean boolFoo(Object... args) {
- return args == null;
- }
-
- /** */
- public static String stringFoo(Object... args) {
- return args == null ? "null" : "not null";
- }
+/**
+ * Tree index interface.
+ *
+ * @param <R> Indexing row type.
+ */
+public interface TreeIndex<R> {
+ /**
+ * Index lookup method.
+ *
+ * @param lower Lower bound.
+ * @param upper Upper bound.
+ * @return Cursor over the rows within bounds.
+ */
+ Cursor<R> find(R lower, R upper);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallImplementor.java
new file mode 100644
index 0000000..6328938
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/CallImplementor.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rex.RexCall;
+
+/**
+ * Implements a call via given translator.
+ *
+ * @see org.apache.calcite.schema.ScalarFunction
+ * @see org.apache.calcite.schema.TableFunction
+ * @see RexImpTable
+ */
+public interface CallImplementor {
+ /**
+ * Implements a call.
+ *
+ * @param translator Translator for the call
+ * @param call Call that should be implemented
+ * @param nullAs The desired mode of {@code null} translation
+ * @return Translated call
+ */
+ Expression implement(
+ RexToLixTranslator translator,
+ RexCall call,
+ RexImpTable.NullAs nullAs);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ConverterUtils.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ConverterUtils.java
new file mode 100644
index 0000000..69cf03c
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ConverterUtils.java
@@ -0,0 +1,460 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.linq4j.tree.UnaryExpression;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
+
+/** */
+public class ConverterUtils {
+ /** */
+ private ConverterUtils() {
+ }
+
+ /**
+ * In Calcite, {@code java.sql.Date} and {@code java.sql.Time} are stored as {@code Integer} type, {@code
+ * java.sql.Timestamp} is stored as {@code Long} type.
+ *
+ * @param operand Operand that should be converted.
+ * @param targetType Required type.
+ *
+ * @return New expression of required type.
+ */
+ static Expression toInternal(Expression operand, Type targetType) {
+ return toInternal(operand, operand.getType(), targetType);
+ }
+
+ /** */
+ private static Expression toInternal(Expression operand,
+ Type fromType, Type targetType) {
+ if (fromType == java.sql.Date.class) {
+ if (targetType == int.class)
+ return Expressions.call(BuiltInMethod.DATE_TO_INT.method, operand);
+ else if (targetType == Integer.class)
+ return Expressions.call(BuiltInMethod.DATE_TO_INT_OPTIONAL.method, operand);
+ }
+ else if (fromType == java.sql.Time.class) {
+ if (targetType == int.class)
+ return Expressions.call(BuiltInMethod.TIME_TO_INT.method, operand);
+ else if (targetType == Integer.class)
+ return Expressions.call(BuiltInMethod.TIME_TO_INT_OPTIONAL.method, operand);
+ }
+ else if (fromType == java.sql.Timestamp.class) {
+ if (targetType == long.class)
+ return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG.method, operand);
+ else if (targetType == Long.class)
+ return Expressions.call(BuiltInMethod.TIMESTAMP_TO_LONG_OPTIONAL.method, operand);
+ }
+ return operand;
+ }
+
+ /** Converts from internal representation to JDBC representation used by
+ * arguments of user-defined functions. For example, converts date values from
+ * {@code int} to {@link java.sql.Date}. */
+ private static Expression fromInternal(Expression operand, Type targetType) {
+ return fromInternal(operand, operand.getType(), targetType);
+ }
+
+ /** */
+ private static Expression fromInternal(Expression operand,
+ Type fromType, Type targetType) {
+ if (operand == ConstantUntypedNull.INSTANCE)
+ return operand;
+ if (!(operand.getType() instanceof Class))
+ return operand;
+ if (Types.isAssignableFrom(targetType, fromType))
+ return operand;
+ if (targetType == java.sql.Date.class) {
+ // E.g. from "int" or "Integer" to "java.sql.Date",
+ // generate "SqlFunctions.internalToDate".
+ if (isA(fromType, Primitive.INT))
+ return Expressions.call(BuiltInMethod.INTERNAL_TO_DATE.method, operand);
+ }
+ else if (targetType == java.sql.Time.class) {
+ // E.g. from "int" or "Integer" to "java.sql.Time",
+ // generate "SqlFunctions.internalToTime".
+ if (isA(fromType, Primitive.INT))
+ return Expressions.call(BuiltInMethod.INTERNAL_TO_TIME.method, operand);
+ }
+ else if (targetType == java.sql.Timestamp.class) {
+ // E.g. from "long" or "Long" to "java.sql.Timestamp",
+ // generate "SqlFunctions.internalToTimestamp".
+ if (isA(fromType, Primitive.LONG))
+ return Expressions.call(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method, operand);
+ }
+ if (Primitive.is(operand.type)
+ && Primitive.isBox(targetType)) {
+ // E.g. operand is "int", target is "Long", generate "(long) operand".
+ return Expressions.convert_(operand,
+ Primitive.ofBox(targetType).primitiveClass);
+ }
+ return operand;
+ }
+
+ /** */
+ static List<Expression> fromInternal(Class<?>[] targetTypes,
+ List<Expression> expressions) {
+ final List<Expression> list = new ArrayList<>();
+ if (targetTypes.length == expressions.size()) {
+ for (int i = 0; i < expressions.size(); i++)
+ list.add(fromInternal(expressions.get(i), targetTypes[i]));
+ } else {
+ int j = 0;
+ for (int i = 0; i < expressions.size(); i++) {
+ Class<?> type;
+ if (!targetTypes[j].isArray()) {
+ type = targetTypes[j];
+ j++;
+ } else
+ type = targetTypes[j].getComponentType();
+
+ list.add(fromInternal(expressions.get(i), type));
+ }
+ }
+ return list;
+ }
+
+ /** */
+ private static Type toInternal(RelDataType type) {
+ return toInternal(type, false);
+ }
+
+ /** */
+ static Type toInternal(RelDataType type, boolean forceNotNull) {
+ switch (type.getSqlTypeName()) {
+ case DATE:
+ case TIME:
+ return type.isNullable() && !forceNotNull ? Integer.class : int.class;
+ case TIMESTAMP:
+ return type.isNullable() && !forceNotNull ? Long.class : long.class;
+ default:
+ return null; // we don't care; use the default storage type
+ }
+ }
+
+ /** */
+ static List<Type> internalTypes(List<? extends RexNode> operandList) {
+ return Util.transform(operandList, node -> toInternal(node.getType()));
+ }
+
+ /**
+ * Convert {@code operand} to target type {@code toType}.
+ *
+ * @param operand The expression to convert
+ * @param toType Target type
+ * @return A new expression with type {@code toType} or original if there is no need to convert
+ */
+ public static Expression convert(Expression operand, Type toType) {
+ final Type fromType = operand.getType();
+ return convert(operand, fromType, toType);
+ }
+
+ /**
+ * Convert {@code operand} to target type {@code toType}.
+ *
+ * @param operand The expression to convert
+ * @param fromType Field type
+ * @param toType Target type
+ * @return A new expression with type {@code toType} or original if there is no need to convert
+ */
+ public static Expression convert(Expression operand, Type fromType, Type toType) {
+ if (!Types.needTypeCast(fromType, toType))
+ return operand;
+ // E.g. from "Short" to "int".
+ // Generate "x.intValue()".
+ final Primitive toPrimitive = Primitive.of(toType);
+ final Primitive toBox = Primitive.ofBox(toType);
+ final Primitive fromBox = Primitive.ofBox(fromType);
+ final Primitive fromPrimitive = Primitive.of(fromType);
+ final boolean fromNumber = fromType instanceof Class
+ && Number.class.isAssignableFrom((Class)fromType);
+ if (fromType == String.class) {
+ if (toPrimitive != null) {
+ switch (toPrimitive) {
+ case CHAR:
+ case SHORT:
+ case INT:
+ case LONG:
+ case FLOAT:
+ case DOUBLE:
+ // Generate "SqlFunctions.toShort(x)".
+ return Expressions.call(
+ SqlFunctions.class,
+ "to" + SqlFunctions.initcap(toPrimitive.primitiveName),
+ operand);
+ default:
+ // Generate "Short.parseShort(x)".
+ return Expressions.call(
+ toPrimitive.boxClass,
+ "parse" + SqlFunctions.initcap(toPrimitive.primitiveName),
+ operand);
+ }
+ }
+ if (toBox != null) {
+ switch (toBox) {
+ case CHAR:
+ // Generate "SqlFunctions.toCharBoxed(x)".
+ return Expressions.call(
+ SqlFunctions.class,
+ "to" + SqlFunctions.initcap(toBox.primitiveName) + "Boxed",
+ operand);
+ default:
+ // Generate "Short.valueOf(x)".
+ return Expressions.call(
+ toBox.boxClass,
+ "valueOf",
+ operand);
+ }
+ }
+ }
+ if (toPrimitive != null) {
+ if (fromPrimitive != null) {
+ // E.g. from "float" to "double"
+ return Expressions.convert_(
+ operand, toPrimitive.primitiveClass);
+ }
+ if (fromNumber || fromBox == Primitive.CHAR) {
+ // Generate "x.shortValue()".
+ return Expressions.unbox(operand, toPrimitive);
+ }
+ else {
+ // E.g. from "Object" to "short".
+ // Generate "SqlFunctions.toShort(x)"
+ return Expressions.call(
+ SqlFunctions.class,
+ "to" + SqlFunctions.initcap(toPrimitive.primitiveName),
+ operand);
+ }
+ }
+ else if (fromNumber && toBox != null) {
+ // E.g. from "Short" to "Integer"
+ // Generate "x == null ? null : Integer.valueOf(x.intValue())"
+ return Expressions.condition(
+ Expressions.equal(operand, RexImpTable.NULL_EXPR),
+ RexImpTable.NULL_EXPR,
+ Expressions.box(
+ Expressions.unbox(operand, toBox),
+ toBox));
+ }
+ else if (fromPrimitive != null && toBox != null) {
+ // E.g. from "int" to "Long".
+ // Generate Long.valueOf(x)
+ // Eliminate primitive casts like Long.valueOf((long) x)
+ if (operand instanceof UnaryExpression) {
+ UnaryExpression una = (UnaryExpression)operand;
+ if (una.nodeType == ExpressionType.Convert
+ && Primitive.of(una.getType()) == toBox) {
+ Primitive origin = Primitive.of(una.expression.type);
+ if (origin != null && toBox.assignableFrom(origin))
+ return Expressions.box(una.expression, toBox);
+ }
+ }
+ if (fromType == toBox.primitiveClass)
+ return Expressions.box(operand, toBox);
+ // E.g., from "int" to "Byte".
+ // Convert it first and generate "Byte.valueOf((byte)x)"
+ // Because there is no method "Byte.valueOf(int)" in Byte
+ return Expressions.box(
+ Expressions.convert_(operand, toBox.primitiveClass),
+ toBox);
+ }
+ // Convert datetime types to internal storage type:
+ // 1. java.sql.Date -> int or Integer
+ // 2. java.sql.Time -> int or Integer
+ // 3. java.sql.Timestamp -> long or Long
+ if (representAsInternalType(fromType)) {
+ final Expression internalTypedOperand =
+ toInternal(operand, fromType, toType);
+ if (operand != internalTypedOperand)
+ return internalTypedOperand;
+ }
+ // Convert internal storage type to datetime types:
+ // 1. int or Integer -> java.sql.Date
+ // 2. int or Integer -> java.sql.Time
+ // 3. long or Long -> java.sql.Timestamp
+ if (representAsInternalType(toType)) {
+ final Expression originTypedOperand =
+ fromInternal(operand, fromType, toType);
+ if (operand != originTypedOperand)
+ return originTypedOperand;
+ }
+ if (toType == BigDecimal.class) {
+ if (fromBox != null) {
+ // E.g. from "Integer" to "BigDecimal".
+ // Generate "x == null ? null : new BigDecimal(x.intValue())"
+ return Expressions.condition(
+ Expressions.equal(operand, RexImpTable.NULL_EXPR),
+ RexImpTable.NULL_EXPR,
+ Expressions.new_(
+ BigDecimal.class,
+ Expressions.unbox(operand, fromBox)));
+ }
+ if (fromPrimitive != null) {
+ // E.g. from "int" to "BigDecimal".
+ // Generate "new BigDecimal(x)"
+ return Expressions.new_(BigDecimal.class, operand);
+ }
+ // E.g. from "Object" to "BigDecimal".
+ // Generate "x == null ? null : SqlFunctions.toBigDecimal(x)"
+ return Expressions.condition(
+ Expressions.equal(operand, RexImpTable.NULL_EXPR),
+ RexImpTable.NULL_EXPR,
+ Expressions.call(
+ SqlFunctions.class,
+ "toBigDecimal",
+ operand));
+ }
+ else if (toType == String.class) {
+ if (fromPrimitive != null) {
+ switch (fromPrimitive) {
+ case DOUBLE:
+ case FLOAT:
+ // E.g. from "double" to "String"
+ // Generate "SqlFunctions.toString(x)"
+ return Expressions.call(
+ SqlFunctions.class,
+ "toString",
+ operand);
+ default:
+ // E.g. from "int" to "String"
+ // Generate "Integer.toString(x)"
+ return Expressions.call(
+ fromPrimitive.boxClass,
+ "toString",
+ operand);
+ }
+ }
+ else if (fromType == BigDecimal.class) {
+ // E.g. from "BigDecimal" to "String"
+ // Generate "SqlFunctions.toString(x)"
+ return Expressions.condition(
+ Expressions.equal(operand, RexImpTable.NULL_EXPR),
+ RexImpTable.NULL_EXPR,
+ Expressions.call(
+ SqlFunctions.class,
+ "toString",
+ operand));
+ }
+ else {
+ Expression result;
+ try {
+ // Avoid to generate code like:
+ // "null.toString()" or "(xxx) null.toString()"
+ if (operand instanceof ConstantExpression) {
+ ConstantExpression ce = (ConstantExpression)operand;
+ if (ce.value == null)
+ return Expressions.convert_(operand, toType);
+ }
+ // Try to call "toString()" method
+ // E.g. from "Integer" to "String"
+ // Generate "x == null ? null : x.toString()"
+ result = Expressions.condition(
+ Expressions.equal(operand, RexImpTable.NULL_EXPR),
+ RexImpTable.NULL_EXPR,
+ Expressions.call(operand, "toString"));
+ }
+ catch (RuntimeException e) {
+ // For some special cases, e.g., "BuiltInMethod.LESSER",
+ // its return type is generic ("Comparable"), which contains
+ // no "toString()" method. We fall through to "(String)x".
+ return Expressions.convert_(operand, toType);
+ }
+ return result;
+ }
+ }
+ return Expressions.convert_(operand, toType);
+ }
+
+ /** */
+ private static boolean isA(Type fromType, Primitive primitive) {
+ return Primitive.of(fromType) == primitive
+ || Primitive.ofBox(fromType) == primitive;
+ }
+
+ /** */
+ private static boolean representAsInternalType(Type type) {
+ return type == java.sql.Date.class
+ || type == java.sql.Time.class
+ || type == java.sql.Timestamp.class;
+ }
+
+ /**
+ * In {@link org.apache.calcite.sql.type.SqlTypeAssignmentRule},
+ * some rules decide whether one type can be assignable to another type.
+ * Based on these rules, a function can accept arguments with assignable types.
+ *
+ * <p>For example, a function with Long type operand can accept Integer as input.
+ * See {@code org.apache.calcite.sql.SqlUtil#filterRoutinesByParameterType()} for details.
+ *
+ * <p>During query execution, some of the assignable types need explicit conversion
+ * to the target types. i.e., Decimal expression should be converted to Integer
+ * before it is assigned to the Integer type Lvalue(In Java, Decimal can not be assigned to
+ * Integer directly).
+ *
+ * @param targetTypes Formal operand types declared for the function arguments
+ * @param arguments Input expressions to the function
+ * @return Input expressions with probable type conversion
+ */
+ static List<Expression> convertAssignableTypes(Class<?>[] targetTypes,
+ List<Expression> arguments) {
+ final List<Expression> list = new ArrayList<>();
+ if (targetTypes.length == arguments.size()) {
+ for (int i = 0; i < arguments.size(); i++)
+ list.add(convertAssignableType(arguments.get(i), targetTypes[i]));
+ } else {
+ int j = 0;
+ for (Expression argument: arguments) {
+ Class<?> type;
+ if (!targetTypes[j].isArray()) {
+ type = targetTypes[j];
+ j++;
+ } else
+ type = targetTypes[j].getComponentType();
+
+ list.add(convertAssignableType(argument, type));
+ }
+ }
+ return list;
+ }
+
+ /**
+ * Handles decimal type specifically with explicit type conversion.
+ */
+ private static Expression convertAssignableType(Expression argument, Type targetType) {
+ if (targetType != BigDecimal.class)
+ return argument;
+
+ return convert(argument, targetType);
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
new file mode 100644
index 0000000..7255955
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactory.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorWrapper;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+
+/**
+ * Expression factory.
+ */
+public interface ExpressionFactory<Row> {
+ /** */
+ Supplier<List<AccumulatorWrapper<Row>>> accumulatorsFactory(
+ AggregateType type,
+ List<AggregateCall> calls,
+ RelDataType rowType
+ );
+
+ /**
+ * Creates a comparator for given data type and collations. Mainly used for sorted exchange.
+ *
+ * @param collations Collations.
+ * @return Row comparator.
+ */
+ Comparator<Row> comparator(RelCollation collations);
+
+ /**
+ * Creates a comparator for different rows by given field collations. Mainly used for merge join rows comparison.
+ * Note: Both list has to have the same size and matched fields collations has to have the same traits
+ * (i.e. all pairs of field collations should have the same sorting and nulls ordering).
+ *
+ * @param left Collations of left row.
+ * @param right Collations of right row.
+ * @return Rows comparator.
+ */
+ Comparator<Row> comparator(List<RelFieldCollation> left, List<RelFieldCollation> right);
+
+ /**
+ * Creates a Filter predicate.
+ * @param filter Filter expression.
+ * @param rowType Input row type.
+ * @return Filter predicate.
+ */
+ Predicate<Row> predicate(RexNode filter, RelDataType rowType);
+
+ /**
+ * Creates a Project function. Resulting function returns a row with different fields,
+ * fields order, fields types, etc.
+ * @param projects Projection expressions.
+ * @param rowType Input row type.
+ * @return Project function.
+ */
+ Function<Row, Row> project(List<RexNode> projects, RelDataType rowType);
+
+ /**
+ * Creates a Values relational node rows source.
+ *
+ * @param values Values.
+ * @param rowType Output row type.
+ * @return Values relational node rows source.
+ */
+ Iterable<Row> values(List<RexLiteral> values, RelDataType rowType);
+
+ /**
+ * Creates row from RexNodes.
+ *
+ * @param values Values.
+ * @return Row.
+ */
+ Supplier<Row> rowSource(List<RexNode> values);
+
+
+ /**
+ * Executes expression.
+ */
+ <T> Supplier<T> execute(RexNode node);
+
+ /**
+ * Creates {@link Scalar}, a code-generated expressions evaluator.
+ *
+ * @param node Expression.
+ * @param type Row type.
+ * @return Scalar.
+ */
+ default Scalar scalar(RexNode node, RelDataType type) {
+ return scalar(ImmutableList.of(node), type);
+ }
+
+ /**
+ * Creates {@link Scalar}, a code-generated expressions evaluator.
+ *
+ * @param nodes Expressions.
+ * @param type Row type.
+ * @return Scalar.
+ */
+ Scalar scalar(List<RexNode> nodes, RelDataType type);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java
new file mode 100644
index 0000000..eb8260d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java
@@ -0,0 +1,509 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Ordering;
+import com.google.common.primitives.Primitives;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.EnumUtils;
+import org.apache.calcite.linq4j.function.Function1;
+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.MethodDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.RexToLixTranslator.InputGetter;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorWrapper;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AccumulatorsFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * Implements rex expression into a function object. Uses JaninoRexCompiler under the hood.
+ * Each expression compiles into a class and a wrapper over it is returned.
+ */
+public class ExpressionFactoryImpl<Row> implements ExpressionFactory<Row> {
+// /** */
+// private static final Map<String, Scalar> SCALAR_CACHE = new GridBoundedConcurrentLinkedHashMap<>(1024);
+
+ /** */
+ private final IgniteTypeFactory typeFactory;
+
+ /** */
+ private final SqlConformance conformance;
+
+ /** */
+ private final RexBuilder rexBuilder;
+
+ /** */
+ private final RelDataType emptyType;
+
+ /** */
+ private final ExecutionContext<Row> ctx;
+
+ /** */
+ public ExpressionFactoryImpl(ExecutionContext<Row> ctx, IgniteTypeFactory typeFactory, SqlConformance conformance) {
+ this.ctx = ctx;
+ this.typeFactory = typeFactory;
+ this.conformance = conformance;
+
+ rexBuilder = new RexBuilder(this.typeFactory);
+ emptyType = new RelDataTypeFactory.Builder(this.typeFactory).build();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Supplier<List<AccumulatorWrapper<Row>>> accumulatorsFactory(
+ AggregateType type,
+ List<AggregateCall> calls,
+ RelDataType rowType
+ ) {
+ if (calls.isEmpty())
+ return null;
+
+ return new AccumulatorsFactory<>(ctx, type, calls, rowType);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Comparator<Row> comparator(RelCollation collation) {
+ if (collation == null || collation.getFieldCollations().isEmpty())
+ return null;
+ else if (collation.getFieldCollations().size() == 1)
+ return comparator(collation.getFieldCollations().get(0));
+ return Ordering.compound(collation.getFieldCollations()
+ .stream()
+ .map(this::comparator)
+ .collect(Collectors.toList()));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Comparator<Row> comparator(List<RelFieldCollation> left, List<RelFieldCollation> right) {
+ if (nullOrEmpty(left) || nullOrEmpty(right) || left.size() != right.size())
+ throw new IllegalArgumentException("Both inputs should be non-empty and have the same size: left="
+ + (left != null ? left.size() : "null") + ", right=" + (right != null ? right.size() : "null"));
+
+ List<Comparator<Row>> comparators = new ArrayList<>();
+
+ for (int i = 0; i < left.size(); i++)
+ comparators.add(comparator(left.get(i), right.get(i)));
+
+ return Ordering.compound(comparators);
+ }
+
+ /** */
+ @SuppressWarnings("rawtypes")
+ private Comparator<Row> comparator(RelFieldCollation fieldCollation) {
+ final int nullComparison = fieldCollation.nullDirection.nullComparison;
+ final int x = fieldCollation.getFieldIndex();
+ RowHandler<Row> handler = ctx.rowHandler();
+
+ if (fieldCollation.direction == RelFieldCollation.Direction.ASCENDING) {
+ return (o1, o2) -> {
+ final Comparable c1 = (Comparable)handler.get(x, o1);
+ final Comparable c2 = (Comparable)handler.get(x, o2);
+ return RelFieldCollation.compare(c1, c2, nullComparison);
+ };
+ }
+
+ return (o1, o2) -> {
+ final Comparable c1 = (Comparable)handler.get(x, o1);
+ final Comparable c2 = (Comparable)handler.get(x, o2);
+ return RelFieldCollation.compare(c2, c1, nullComparison);
+ };
+ }
+
+ /** */
+ @SuppressWarnings("rawtypes")
+ private Comparator<Row> comparator(RelFieldCollation left, RelFieldCollation right) {
+ final int nullComparison = left.nullDirection.nullComparison;
+
+ if (nullComparison != right.nullDirection.nullComparison)
+ throw new IllegalArgumentException("Can't be compared: left=" + left + ", right=" + right);
+
+ final int lIdx = left.getFieldIndex();
+ final int rIdx = right.getFieldIndex();
+ RowHandler<Row> handler = ctx.rowHandler();
+
+ if (left.direction != right.direction)
+ throw new IllegalArgumentException("Can't be compared: left=" + left + ", right=" + right);
+
+ if (left.direction == RelFieldCollation.Direction.ASCENDING) {
+ return (o1, o2) -> {
+ final Comparable c1 = (Comparable)handler.get(lIdx, o1);
+ final Comparable c2 = (Comparable)handler.get(rIdx, o2);
+ return RelFieldCollation.compare(c1, c2, nullComparison);
+ };
+ }
+
+ return (o1, o2) -> {
+ final Comparable c1 = (Comparable)handler.get(lIdx, o1);
+ final Comparable c2 = (Comparable)handler.get(rIdx, o2);
+ return RelFieldCollation.compare(c2, c1, -nullComparison);
+ };
+ }
+
+ /** {@inheritDoc} */
+ @Override public Predicate<Row> predicate(RexNode filter, RelDataType rowType) {
+ return new PredicateImpl(scalar(filter, rowType));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Function<Row, Row> project(List<RexNode> projects, RelDataType rowType) {
+ return new ProjectImpl(scalar(projects, rowType), ctx.rowHandler().factory(typeFactory, RexUtil.types(projects)));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Supplier<Row> rowSource(List<RexNode> values) {
+ return new ValuesImpl(scalar(values, null), ctx.rowHandler().factory(typeFactory, RexUtil.types(values)));
+ }
+
+ /** {@inheritDoc} */
+ @Override public <T> Supplier<T> execute(RexNode node) {
+ return new ValueImpl<T>(scalar(node, null), ctx.rowHandler().factory(typeFactory.getJavaClass(node.getType())));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Iterable<Row> values(List<RexLiteral> values, RelDataType rowType) {
+ RowHandler<Row> handler = ctx.rowHandler();
+ RowFactory<Row> factory = handler.factory(typeFactory, rowType);
+
+ int columns = rowType.getFieldCount();
+ assert values.size() % columns == 0;
+
+ List<Class<?>> types = new ArrayList<>(columns);
+ for (RelDataType type : RelOptUtil.getFieldTypeList(rowType))
+ types.add(Primitives.wrap((Class<?>)typeFactory.getJavaClass(type)));
+
+ List<Row> rows = new ArrayList<>(values.size() / columns);
+ Row currRow = null;
+ for (int i = 0; i < values.size(); i++) {
+ int field = i % columns;
+
+ if (field == 0)
+ rows.add(currRow = factory.create());
+
+ RexLiteral literal = values.get(i);
+
+ handler.set(field, currRow, literal.getValueAs(types.get(field)));
+ }
+
+ return rows;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Scalar scalar(List<RexNode> nodes, RelDataType type) {
+ return compile(nodes, type);
+ }
+
+ /** */
+ private Scalar compile(Iterable<RexNode> nodes, RelDataType type) {
+ if (type == null)
+ type = emptyType;
+
+ RexProgramBuilder programBuilder = new RexProgramBuilder(type, rexBuilder);
+
+ for (RexNode node : nodes)
+ programBuilder.addProject(node, null);
+
+ RexProgram program = programBuilder.getProgram();
+
+ BlockBuilder builder = new BlockBuilder();
+
+ ParameterExpression ctx_ =
+ Expressions.parameter(ExecutionContext.class, "ctx");
+
+ ParameterExpression in_ =
+ Expressions.parameter(Object.class, "in");
+
+ ParameterExpression out_ =
+ Expressions.parameter(Object.class, "out");
+
+ builder.add(
+ Expressions.declare(Modifier.FINAL, DataContext.ROOT, Expressions.convert_(ctx_, DataContext.class)));
+
+ Expression hnd_ = builder.append("hnd",
+ Expressions.call(ctx_,
+ IgniteMethod.CONTEXT_ROW_HANDLER.method()));
+
+ InputGetter inputGetter = new FieldGetter(hnd_, in_, type);
+
+ Function1<String, InputGetter> correlates = new CorrelatesBuilder(builder, ctx_, hnd_).build(nodes);
+
+ List<Expression> projects = RexToLixTranslator.translateProjects(program, typeFactory, conformance,
+ builder, null, ctx_, inputGetter, correlates);
+
+ for (int i = 0; i < projects.size(); i++) {
+ builder.add(
+ Expressions.statement(
+ Expressions.call(hnd_,
+ IgniteMethod.ROW_HANDLER_SET.method(),
+ Expressions.constant(i), out_, projects.get(i))));
+ }
+
+ MethodDeclaration decl = Expressions.methodDecl(
+ Modifier.PUBLIC, void.class, IgniteMethod.SCALAR_EXECUTE.method().getName(),
+ ImmutableList.of(ctx_, in_, out_), builder.toBlock());
+
+ return Commons.compile(Scalar.class, Expressions.toString(List.of(decl), "\n", false));
+ }
+
+ /** */
+ private String digest(List<RexNode> nodes, RelDataType type) {
+ StringBuilder b = new StringBuilder();
+
+ b.append('[');
+
+ for (int i = 0; i < nodes.size(); i++) {
+ if (i > 0)
+ b.append(';');
+
+ b.append(nodes.get(i));
+ }
+
+ b.append(']');
+
+ if (type != null)
+ b.append(':').append(type.getFullTypeString());
+
+ return b.toString();
+ }
+
+ /** */
+ private class PredicateImpl implements Predicate<Row> {
+ /** */
+ private final Scalar scalar;
+
+ /** */
+ private final Row out;
+
+ /** */
+ private final RowHandler<Row> handler;
+
+ /**
+ * @param scalar Scalar.
+ */
+ private PredicateImpl(Scalar scalar) {
+ this.scalar = scalar;
+ handler = ctx.rowHandler();
+ out = handler.factory(typeFactory, typeFactory.createJavaType(Boolean.class)).create();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean test(Row r) {
+ scalar.execute(ctx, r, out);
+
+ return Boolean.TRUE == handler.get(0, out);
+ }
+ }
+
+ /** */
+ private class ProjectImpl implements Function<Row, Row> {
+ /** */
+ private final Scalar scalar;
+
+ /** */
+ private final RowFactory<Row> factory;
+
+ /**
+ * @param scalar Scalar.
+ * @param factory Row factory.
+ */
+ private ProjectImpl(Scalar scalar, RowFactory<Row> factory) {
+ this.scalar = scalar;
+ this.factory = factory;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Row apply(Row r) {
+ Row res = factory.create();
+ scalar.execute(ctx, r, res);
+
+ return res;
+ }
+ }
+
+ /** */
+ private class ValuesImpl implements Supplier<Row> {
+ /** */
+ private final Scalar scalar;
+
+ /** */
+ private final RowFactory<Row> factory;
+
+ /** */
+ private ValuesImpl(Scalar scalar, RowFactory<Row> factory) {
+ this.scalar = scalar;
+ this.factory = factory;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Row get() {
+ Row res = factory.create();
+ scalar.execute(ctx, null, res);
+
+ return res;
+ }
+ }
+
+ /** */
+ private class ValueImpl<T> implements Supplier<T> {
+ /** */
+ private final Scalar scalar;
+
+ /** */
+ private final RowFactory<Row> factory;
+
+ /** */
+ private ValueImpl(Scalar scalar, RowFactory<Row> factory) {
+ this.scalar = scalar;
+ this.factory = factory;
+ }
+
+ /** {@inheritDoc} */
+ @Override public T get() {
+ Row res = factory.create();
+ scalar.execute(ctx, null, res);
+
+ return (T)ctx.rowHandler().get(0, res);
+ }
+ }
+
+ /** */
+ private class FieldGetter implements InputGetter {
+ /** */
+ private final Expression hnd_;
+
+ /** */
+ private final Expression row_;
+
+ /** */
+ private final RelDataType rowType;
+
+ /** */
+ private FieldGetter(Expression hnd_, Expression row_, RelDataType rowType) {
+ this.hnd_ = hnd_;
+ this.row_ = row_;
+ this.rowType = rowType;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Expression field(BlockBuilder list, int index, Type desiredType) {
+ Expression row_ = list.append("row", this.row_);
+
+ Expression field = Expressions.call(hnd_,
+ IgniteMethod.ROW_HANDLER_GET.method(),
+ Expressions.constant(index), row_);
+
+ Type fieldType = typeFactory.getJavaClass(rowType.getFieldList().get(index).getType());
+
+ if (desiredType == null) {
+ desiredType = fieldType;
+ fieldType = Object.class;
+ } else if (fieldType != java.sql.Date.class
+ && fieldType != java.sql.Time.class
+ && fieldType != java.sql.Timestamp.class) {
+ fieldType = Object.class;
+ }
+
+ return EnumUtils.convert(field, fieldType, desiredType);
+ }
+ }
+
+ /** */
+ private class CorrelatesBuilder extends RexShuttle {
+ /** */
+ private final BlockBuilder builder;
+
+ /** */
+ private final Expression ctx_;
+
+ /** */
+ private final Expression hnd_;
+
+ /** */
+ private Map<String, FieldGetter> correlates;
+
+ /** */
+ private CorrelatesBuilder(BlockBuilder builder, Expression ctx_, Expression hnd_) {
+ this.builder = builder;
+ this.hnd_ = hnd_;
+ this.ctx_ = ctx_;
+ }
+
+ /** */
+ public Function1<String, InputGetter> build(Iterable<RexNode> nodes) {
+ try {
+ for (RexNode node : nodes)
+ node.accept(this);
+
+ return correlates == null ? null : correlates::get;
+ }
+ finally {
+ correlates = null;
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public RexNode visitCorrelVariable(RexCorrelVariable variable) {
+ Expression corr_ = builder.append("corr",
+ Expressions.call(ctx_, IgniteMethod.CONTEXT_GET_CORRELATED_VALUE.method(),
+ Expressions.constant(variable.id.getId())));
+
+ if (correlates == null)
+ correlates = new HashMap<>();
+
+ correlates.put(variable.getName(), new FieldGetter(hnd_, corr_, variable.getType()));
+
+ return variable;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteBuiltInMethod.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteBuiltInMethod.java
new file mode 100644
index 0000000..bbcc881
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteBuiltInMethod.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Method;
+
+import org.apache.calcite.linq4j.tree.Types;
+
+/**
+ * Built-in methods.
+ */
+public enum IgniteBuiltInMethod {
+ /** */
+ SYSTEM_RANGE2(IgniteSqlFunctions.class, "systemRange", Object.class, Object.class),
+
+ /** */
+ SYSTEM_RANGE3(IgniteSqlFunctions.class, "systemRange", Object.class, Object.class, Object.class);
+
+ /** */
+ public final Method method;
+
+ /** */
+ IgniteBuiltInMethod(Method method) {
+ this.method = method;
+ }
+
+ /** Defines a method. */
+ IgniteBuiltInMethod(Class clazz, String methodName, Class... argumentTypes) {
+ this(Types.lookupMethod(clazz, methodName, argumentTypes));
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteSqlFunctions.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteSqlFunctions.java
new file mode 100644
index 0000000..203c676
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/IgniteSqlFunctions.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.ScannableTable;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Ignite SQL functions.
+ */
+public class IgniteSqlFunctions {
+ /**
+ * Default constructor.
+ */
+ private IgniteSqlFunctions() {
+ // No-op.
+ }
+
+ /** SQL SYSTEM_RANGE(start, end) table function. */
+ public static ScannableTable systemRange(Object rangeStart, Object rangeEnd) {
+ return new RangeTable(rangeStart, rangeEnd, 1L);
+ }
+
+ /** SQL SYSTEM_RANGE(start, end, increment) table function. */
+ public static ScannableTable systemRange(Object rangeStart, Object rangeEnd, Object increment) {
+ return new RangeTable(rangeStart, rangeEnd, increment);
+ }
+
+ /** */
+ private static class RangeTable implements ScannableTable {
+ /** Start of the range. */
+ private final Object rangeStart;
+
+ /** End of the range. */
+ private final Object rangeEnd;
+
+ /** Increment. */
+ private final Object increment;
+
+ /**
+ * Note: {@code Object} arguments required here due to:
+ * 1. {@code NULL} arguments need to be supported, so we can't use {@code long} arguments type.
+ * 2. {@code Integer} and other numeric classes can be converted to {@code long} type by java, but can't be
+ * converted to {@code Long} type, so we can't use {@code Long} arguments type either.
+ * Instead, we accept {@code Object} arguments type and try to convert valid types to {@code long}.
+ */
+ RangeTable(Object rangeStart, Object rangeEnd, Object increment) {
+ this.rangeStart = rangeStart;
+ this.rangeEnd = rangeEnd;
+ this.increment = increment;
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+ return typeFactory.builder().add("X", SqlTypeName.BIGINT).build();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Enumerable<@Nullable Object[]> scan(DataContext root) {
+ if (rangeStart == null || rangeEnd == null || increment == null)
+ return Linq4j.emptyEnumerable();
+
+ long rangeStart = convertToLongArg(this.rangeStart, "rangeStart");
+ long rangeEnd = convertToLongArg(this.rangeEnd, "rangeEnd");
+ long increment = convertToLongArg(this.increment, "increment");
+
+ if (increment == 0L)
+ throw new IllegalArgumentException("Increment can't be 0");
+
+ return new AbstractEnumerable<@Nullable Object[]>() {
+ @Override public Enumerator<@Nullable Object[]> enumerator() {
+ return new Enumerator<Object[]>() {
+ long cur = rangeStart - increment;
+
+ @Override public Object[] current() {
+ return new Object[] { cur };
+ }
+
+ @Override public boolean moveNext() {
+ cur += increment;
+
+ return increment > 0L ? cur <= rangeEnd : cur >= rangeEnd;
+ }
+
+ @Override public void reset() {
+ cur = rangeStart - increment;
+ }
+
+ @Override public void close() {
+ // No-op.
+ }
+ };
+ }
+ };
+ }
+
+ /** */
+ private long convertToLongArg(Object val, String name) {
+ if (val instanceof Byte || val instanceof Short || val instanceof Integer || val instanceof Long)
+ return ((Number)val).longValue();
+
+ throw new IllegalArgumentException("Unsupported argument type [arg=" + name +
+ ", type=" + val.getClass().getSimpleName() + ']');
+ }
+
+ /** {@inheritDoc} */
+ @Override public Statistic getStatistic() {
+ throw new UnsupportedOperationException();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Schema.TableType getJdbcTableType() {
+ return Schema.TableType.TABLE;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isRolledUp(String column) {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean rolledUpColumnValidInsideAgg(String column, SqlCall call,
+ SqlNode parent, CalciteConnectionConfig cfg) {
+ return true;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ImplementableFunction.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ImplementableFunction.java
new file mode 100644
index 0000000..a30c01f
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ImplementableFunction.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.ScalarFunction;
+import org.apache.calcite.schema.TableFunction;
+
+/**
+ * Function that can be translated to java code.
+ *
+ * @see ScalarFunction
+ * @see TableFunction
+ */
+public interface ImplementableFunction extends Function {
+ /**
+ * Returns implementor that translates the function to linq4j expression.
+ *
+ * @return implementor that translates the function to linq4j expression.
+ */
+ CallImplementor getImplementor();
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/NotNullImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/NotNullImplementor.java
new file mode 100644
index 0000000..2ff3061
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/NotNullImplementor.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.util.List;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rex.RexCall;
+
+/**
+ * Simplified version of {@link CallImplementor} that does not know about null semantics.
+ *
+ * @see RexImpTable
+ * @see CallImplementor
+ */
+public interface NotNullImplementor {
+ /**
+ * Implements a call with assumption that all the null-checking is implemented by caller.
+ *
+ * @param translator translator to implement the code
+ * @param call call to implement
+ * @param translatedOperands arguments of a call
+ * @return expression that implements given call
+ */
+ Expression implement(
+ RexToLixTranslator translator,
+ RexCall call,
+ List<Expression> translatedOperands);
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ReflectiveCallNotNullImplementor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ReflectiveCallNotNullImplementor.java
new file mode 100644
index 0000000..857605b
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ReflectiveCallNotNullImplementor.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.List;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.rex.RexCall;
+
+/**
+ * Implementation of {@link NotNullImplementor} that calls a given {@link Method}.
+ *
+ * <p>When method is not static, a new instance of the required class is
+ * created.
+ */
+public class ReflectiveCallNotNullImplementor implements NotNullImplementor {
+ /** */
+ protected final Method method;
+
+ /**
+ * Constructor of ReflectiveCallNotNullImplementor.
+ *
+ * @param method Method that is used to implement the call
+ */
+ public ReflectiveCallNotNullImplementor(Method method) {
+ this.method = method;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Expression implement(RexToLixTranslator translator,
+ RexCall call, List<Expression> translatedOperands) {
+ translatedOperands =
+ ConverterUtils.fromInternal(method.getParameterTypes(), translatedOperands);
+ translatedOperands =
+ ConverterUtils.convertAssignableTypes(method.getParameterTypes(), translatedOperands);
+ final Expression callExpr;
+ if ((method.getModifiers() & Modifier.STATIC) != 0)
+ callExpr = Expressions.call(method, translatedOperands);
+
+ else {
+ // The UDF class must have a public zero-args constructor.
+ // Assume that the validator checked already.
+ final Expression target =
+ Expressions.new_(method.getDeclaringClass());
+ callExpr = Expressions.call(target, method, translatedOperands);
+ }
+ if (!containsCheckedException(method))
+ return callExpr;
+
+ return translator.handleMethodCheckedExceptions(callExpr);
+ }
+
+ /** */
+ private boolean containsCheckedException(Method method) {
+ Class[] exceptions = method.getExceptionTypes();
+ if (exceptions == null || exceptions.length == 0)
+ return false;
+
+ for (Class clazz : exceptions) {
+ if (!RuntimeException.class.isAssignableFrom(clazz))
+ return true;
+ }
+ return false;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexImpTable.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexImpTable.java
new file mode 100644
index 0000000..bf91db0
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexImpTable.java
@@ -0,0 +1,2574 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.adapter.enumerable.EnumUtils;
+import org.apache.calcite.adapter.enumerable.NullPolicy;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.ExpressionType;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MemberExpression;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
+import org.apache.calcite.linq4j.tree.OptimizeShuttle;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.schema.QueryableTable;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlJsonEmptyOrError;
+import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.SqlTypeConstructorFunction;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTrimFunction;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.Util;
+import org.apache.ignite.internal.processors.query.calcite.util.IgniteMethod;
+
+import static org.apache.calcite.adapter.enumerable.EnumUtils.generateCollatorExpression;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Add;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Divide;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Equal;
+import static org.apache.calcite.linq4j.tree.ExpressionType.GreaterThan;
+import static org.apache.calcite.linq4j.tree.ExpressionType.GreaterThanOrEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.LessThan;
+import static org.apache.calcite.linq4j.tree.ExpressionType.LessThanOrEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Multiply;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Negate;
+import static org.apache.calcite.linq4j.tree.ExpressionType.NotEqual;
+import static org.apache.calcite.linq4j.tree.ExpressionType.Subtract;
+import static org.apache.calcite.linq4j.tree.ExpressionType.UnaryPlus;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.CHR;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.COMPRESS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.CONCAT2;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.CONCAT_FUNCTION;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.COSH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DATE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DATE_FROM_UNIX_DATE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DAYNAME;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DIFFERENCE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.EXISTS_NODE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.EXTRACT_VALUE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.EXTRACT_XML;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.FROM_BASE64;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_DEPTH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_KEYS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_LENGTH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_PRETTY;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_REMOVE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_STORAGE_SIZE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_TYPE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.LEFT;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.MD5;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.MONTHNAME;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.REGEXP_REPLACE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.REPEAT;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.REVERSE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.RIGHT;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SHA1;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SINH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SOUNDEX;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.SPACE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.STRCMP;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TANH;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TIMESTAMP_MICROS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TIMESTAMP_MILLIS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TIMESTAMP_SECONDS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TO_BASE64;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.TRANSLATE3;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.UNIX_DATE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.UNIX_MICROS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.UNIX_MILLIS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.UNIX_SECONDS;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.XML_TRANSFORM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ABS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ACOS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.AND;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ASCII;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ASIN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ATAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ATAN2;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CARDINALITY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CAST;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CBRT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CEIL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHARACTER_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CHAR_LENGTH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COALESCE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CONCAT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_CATALOG;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_DATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_PATH;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_ROLE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_TIME;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_TIMESTAMP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CURRENT_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DATETIME_PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DEFAULT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DEGREES;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.DIVIDE_INTEGER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ELEMENT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EQUALS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.EXTRACT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.FLOOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.INITCAP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_A_SET;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_EMPTY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_FALSE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_JSON_ARRAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_JSON_OBJECT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_JSON_SCALAR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_JSON_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_A_SET;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_EMPTY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_FALSE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_JSON_ARRAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_JSON_OBJECT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_JSON_SCALAR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_JSON_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NOT_TRUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_TRUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ITEM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_EXISTS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_QUERY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_VALUE_EXPRESSION;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LAST_DAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LIKE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOCALTIME;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOCALTIMESTAMP;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOG10;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.LOWER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MEMBER_OF;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MINUS_DATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MOD;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTIPLY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_EXCEPT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_EXCEPT_DISTINCT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_INTERSECT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_INTERSECT_DISTINCT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_UNION;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.MULTISET_UNION_DISTINCT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NEXT_VALUE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_EQUALS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_LIKE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_SIMILAR_TO;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.NOT_SUBMULTISET_OF;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OR;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.OVERLAY;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PI;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.POSITION;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.POWER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RADIANS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RAND;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.RAND_INTEGER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.REINTERPRET;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.REPLACE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ROUND;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ROW;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SESSION_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SIGN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SIMILAR_TO;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SIN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SLICE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.STRUCT_ACCESS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUBMULTISET_OF;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUBSTRING;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SYSTEM_USER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TRIM;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TRUNCATE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UNARY_MINUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UNARY_PLUS;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.UPPER;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.USER;
+import static org.apache.ignite.internal.processors.query.calcite.sql.fun.IgniteSqlOperatorTable.LENGTH;
+import static org.apache.ignite.internal.processors.query.calcite.sql.fun.IgniteSqlOperatorTable.SYSTEM_RANGE;
+
+/**
+ * Contains implementations of Rex operators as Java code.
+ */
+public class RexImpTable {
+ /** */
+ public static final RexImpTable INSTANCE = new RexImpTable();
+
+ /** */
+ public static final ConstantExpression NULL_EXPR = Expressions.constant(null);
+
+ /** */
+ public static final ConstantExpression FALSE_EXPR = Expressions.constant(false);
+
+ /** */
+ public static final ConstantExpression TRUE_EXPR = Expressions.constant(true);
+
+ /** */
+ public static final MemberExpression BOXED_FALSE_EXPR = Expressions.field(null, Boolean.class, "FALSE");
+
+ /** */
+ public static final MemberExpression BOXED_TRUE_EXPR = Expressions.field(null, Boolean.class, "TRUE");
+
+ /** */
+ private final Map<SqlOperator, RexCallImplementor> map = new HashMap<>();
+
+ /** */
+ RexImpTable() {
+ defineMethod(ROW, BuiltInMethod.ARRAY.method, NullPolicy.NONE);
+ defineMethod(UPPER, BuiltInMethod.UPPER.method, NullPolicy.STRICT);
+ defineMethod(LOWER, BuiltInMethod.LOWER.method, NullPolicy.STRICT);
+ defineMethod(INITCAP, BuiltInMethod.INITCAP.method, NullPolicy.STRICT);
+ defineMethod(TO_BASE64, BuiltInMethod.TO_BASE64.method, NullPolicy.STRICT);
+ defineMethod(FROM_BASE64, BuiltInMethod.FROM_BASE64.method, NullPolicy.STRICT);
+ defineMethod(MD5, BuiltInMethod.MD5.method, NullPolicy.STRICT);
+ defineMethod(SHA1, BuiltInMethod.SHA1.method, NullPolicy.STRICT);
+ defineMethod(SUBSTRING, BuiltInMethod.SUBSTRING.method, NullPolicy.STRICT);
+ defineMethod(LEFT, BuiltInMethod.LEFT.method, NullPolicy.ANY);
+ defineMethod(RIGHT, BuiltInMethod.RIGHT.method, NullPolicy.ANY);
+ defineMethod(REPLACE, BuiltInMethod.REPLACE.method, NullPolicy.STRICT);
+ defineMethod(TRANSLATE3, BuiltInMethod.TRANSLATE3.method, NullPolicy.STRICT);
+ defineMethod(CHR, "chr", NullPolicy.STRICT);
+ defineMethod(CHARACTER_LENGTH, BuiltInMethod.CHAR_LENGTH.method, NullPolicy.STRICT);
+ defineMethod(CHAR_LENGTH, BuiltInMethod.CHAR_LENGTH.method, NullPolicy.STRICT);
+ defineMethod(LENGTH, BuiltInMethod.CHAR_LENGTH.method, NullPolicy.STRICT);
+ defineMethod(CONCAT, BuiltInMethod.STRING_CONCAT.method, NullPolicy.STRICT);
+ defineMethod(CONCAT_FUNCTION, BuiltInMethod.MULTI_STRING_CONCAT.method, NullPolicy.STRICT);
+ defineMethod(CONCAT2, BuiltInMethod.STRING_CONCAT.method, NullPolicy.STRICT);
+ defineMethod(OVERLAY, BuiltInMethod.OVERLAY.method, NullPolicy.STRICT);
+ defineMethod(POSITION, BuiltInMethod.POSITION.method, NullPolicy.STRICT);
+ defineMethod(ASCII, BuiltInMethod.ASCII.method, NullPolicy.STRICT);
+ defineMethod(REPEAT, BuiltInMethod.REPEAT.method, NullPolicy.STRICT);
+ defineMethod(SPACE, BuiltInMethod.SPACE.method, NullPolicy.STRICT);
+ defineMethod(STRCMP, BuiltInMethod.STRCMP.method, NullPolicy.STRICT);
+ defineMethod(SOUNDEX, BuiltInMethod.SOUNDEX.method, NullPolicy.STRICT);
+ defineMethod(DIFFERENCE, BuiltInMethod.DIFFERENCE.method, NullPolicy.STRICT);
+ defineMethod(REVERSE, BuiltInMethod.REVERSE.method, NullPolicy.STRICT);
+
+ map.put(TRIM, new TrimImplementor());
+
+ // logical
+ map.put(AND, new LogicalAndImplementor());
+ map.put(OR, new LogicalOrImplementor());
+ map.put(NOT, new LogicalNotImplementor());
+
+ // comparisons
+ defineBinary(LESS_THAN, LessThan, NullPolicy.STRICT, "lt");
+ defineBinary(LESS_THAN_OR_EQUAL, LessThanOrEqual, NullPolicy.STRICT, "le");
+ defineBinary(GREATER_THAN, GreaterThan, NullPolicy.STRICT, "gt");
+ defineBinary(GREATER_THAN_OR_EQUAL, GreaterThanOrEqual, NullPolicy.STRICT,
+ "ge");
+ defineBinary(EQUALS, Equal, NullPolicy.STRICT, "eq");
+ defineBinary(NOT_EQUALS, NotEqual, NullPolicy.STRICT, "ne");
+
+ // arithmetic
+ defineBinary(PLUS, Add, NullPolicy.STRICT, "plus");
+ defineBinary(MINUS, Subtract, NullPolicy.STRICT, "minus");
+ defineBinary(MULTIPLY, Multiply, NullPolicy.STRICT, "multiply");
+ defineBinary(DIVIDE, Divide, NullPolicy.STRICT, "divide");
+ defineBinary(DIVIDE_INTEGER, Divide, NullPolicy.STRICT, "divide");
+ defineUnary(UNARY_MINUS, Negate, NullPolicy.STRICT,
+ BuiltInMethod.BIG_DECIMAL_NEGATE.getMethodName());
+ defineUnary(UNARY_PLUS, UnaryPlus, NullPolicy.STRICT, null);
+
+ defineMethod(MOD, "mod", NullPolicy.STRICT);
+ defineMethod(EXP, "exp", NullPolicy.STRICT);
+ defineMethod(POWER, "power", NullPolicy.STRICT);
+ defineMethod(LN, "ln", NullPolicy.STRICT);
+ defineMethod(LOG10, "log10", NullPolicy.STRICT);
+ defineMethod(ABS, "abs", NullPolicy.STRICT);
+
+ map.put(RAND, new RandImplementor());
+ map.put(RAND_INTEGER, new RandIntegerImplementor());
+
+ defineMethod(ACOS, "acos", NullPolicy.STRICT);
+ defineMethod(ASIN, "asin", NullPolicy.STRICT);
+ defineMethod(ATAN, "atan", NullPolicy.STRICT);
+ defineMethod(ATAN2, "atan2", NullPolicy.STRICT);
+ defineMethod(CBRT, "cbrt", NullPolicy.STRICT);
+ defineMethod(COS, "cos", NullPolicy.STRICT);
+ defineMethod(COSH, "cosh", NullPolicy.STRICT);
+ defineMethod(COT, "cot", NullPolicy.STRICT);
+ defineMethod(DEGREES, "degrees", NullPolicy.STRICT);
+ defineMethod(RADIANS, "radians", NullPolicy.STRICT);
+ defineMethod(ROUND, "sround", NullPolicy.STRICT);
+ defineMethod(SIGN, "sign", NullPolicy.STRICT);
+ defineMethod(SIN, "sin", NullPolicy.STRICT);
+ defineMethod(SINH, "sinh", NullPolicy.STRICT);
+ defineMethod(TAN, "tan", NullPolicy.STRICT);
+ defineMethod(TANH, "tanh", NullPolicy.STRICT);
+ defineMethod(TRUNCATE, "struncate", NullPolicy.STRICT);
+
+ map.put(PI, new PiImplementor());
+
+ // datetime
+ map.put(DATETIME_PLUS, new DatetimeArithmeticImplementor());
+ map.put(MINUS_DATE, new DatetimeArithmeticImplementor());
+ map.put(EXTRACT, new ExtractImplementor());
+ map.put(FLOOR,
+ new FloorImplementor(BuiltInMethod.FLOOR.method.getName(),
+ BuiltInMethod.UNIX_TIMESTAMP_FLOOR.method,
+ BuiltInMethod.UNIX_DATE_FLOOR.method));
+ map.put(CEIL,
+ new FloorImplementor(BuiltInMethod.CEIL.method.getName(),
+ BuiltInMethod.UNIX_TIMESTAMP_CEIL.method,
+ BuiltInMethod.UNIX_DATE_CEIL.method));
+
+ defineMethod(LAST_DAY, "lastDay", NullPolicy.STRICT);
+ map.put(DAYNAME,
+ new PeriodNameImplementor("dayName",
+ BuiltInMethod.DAYNAME_WITH_TIMESTAMP,
+ BuiltInMethod.DAYNAME_WITH_DATE));
+ map.put(MONTHNAME,
+ new PeriodNameImplementor("monthName",
+ BuiltInMethod.MONTHNAME_WITH_TIMESTAMP,
+ BuiltInMethod.MONTHNAME_WITH_DATE));
+ defineMethod(TIMESTAMP_SECONDS, "timestampSeconds", NullPolicy.STRICT);
+ defineMethod(TIMESTAMP_MILLIS, "timestampMillis", NullPolicy.STRICT);
+ defineMethod(TIMESTAMP_MICROS, "timestampMicros", NullPolicy.STRICT);
+ defineMethod(UNIX_SECONDS, "unixSeconds", NullPolicy.STRICT);
+ defineMethod(UNIX_MILLIS, "unixMillis", NullPolicy.STRICT);
+ defineMethod(UNIX_MICROS, "unixMicros", NullPolicy.STRICT);
+ defineMethod(DATE_FROM_UNIX_DATE, "dateFromUnixDate", NullPolicy.STRICT);
+ defineMethod(UNIX_DATE, "unixDate", NullPolicy.STRICT);
+
+ map.put(IS_NULL, new IsNullImplementor());
+ map.put(IS_NOT_NULL, new IsNotNullImplementor());
+ map.put(IS_TRUE, new IsTrueImplementor());
+ map.put(IS_NOT_TRUE, new IsNotTrueImplementor());
+ map.put(IS_FALSE, new IsFalseImplementor());
+ map.put(IS_NOT_FALSE, new IsNotFalseImplementor());
+
+ // LIKE and SIMILAR
+ final MethodImplementor likeImplementor =
+ new MethodImplementor(BuiltInMethod.LIKE.method, NullPolicy.STRICT,
+ false);
+ map.put(LIKE, likeImplementor);
+ map.put(NOT_LIKE, likeImplementor);
+ final MethodImplementor similarImplementor =
+ new MethodImplementor(BuiltInMethod.SIMILAR.method, NullPolicy.STRICT,
+ false);
+ map.put(SIMILAR_TO, similarImplementor);
+ map.put(NOT_SIMILAR_TO, NotImplementor.of(similarImplementor));
+
+ // POSIX REGEX
+ final MethodImplementor posixRegexImplementor =
+ new MethodImplementor(BuiltInMethod.POSIX_REGEX.method,
+ NullPolicy.STRICT, false);
+ map.put(SqlStdOperatorTable.POSIX_REGEX_CASE_INSENSITIVE,
+ posixRegexImplementor);
+ map.put(SqlStdOperatorTable.POSIX_REGEX_CASE_SENSITIVE,
+ posixRegexImplementor);
+ map.put(SqlStdOperatorTable.NEGATED_POSIX_REGEX_CASE_INSENSITIVE,
+ NotImplementor.of(posixRegexImplementor));
+ map.put(SqlStdOperatorTable.NEGATED_POSIX_REGEX_CASE_SENSITIVE,
+ NotImplementor.of(posixRegexImplementor));
+ map.put(REGEXP_REPLACE, new RegexpReplaceImplementor());
+
+ // Multisets & arrays
+ defineMethod(CARDINALITY, BuiltInMethod.COLLECTION_SIZE.method,
+ NullPolicy.STRICT);
+ defineMethod(SLICE, BuiltInMethod.SLICE.method, NullPolicy.NONE);
+ defineMethod(ELEMENT, BuiltInMethod.ELEMENT.method, NullPolicy.STRICT);
+ defineMethod(STRUCT_ACCESS, BuiltInMethod.STRUCT_ACCESS.method, NullPolicy.ANY);
+ defineMethod(MEMBER_OF, BuiltInMethod.MEMBER_OF.method, NullPolicy.NONE);
+ final MethodImplementor isEmptyImplementor =
+ new MethodImplementor(BuiltInMethod.IS_EMPTY.method, NullPolicy.NONE,
+ false);
+ map.put(IS_EMPTY, isEmptyImplementor);
+ map.put(IS_NOT_EMPTY, NotImplementor.of(isEmptyImplementor));
+ final MethodImplementor isASetImplementor =
+ new MethodImplementor(BuiltInMethod.IS_A_SET.method, NullPolicy.NONE,
+ false);
+ map.put(IS_A_SET, isASetImplementor);
+ map.put(IS_NOT_A_SET, NotImplementor.of(isASetImplementor));
+ defineMethod(MULTISET_INTERSECT_DISTINCT,
+ BuiltInMethod.MULTISET_INTERSECT_DISTINCT.method, NullPolicy.NONE);
+ defineMethod(MULTISET_INTERSECT,
+ BuiltInMethod.MULTISET_INTERSECT_ALL.method, NullPolicy.NONE);
+ defineMethod(MULTISET_EXCEPT_DISTINCT,
+ BuiltInMethod.MULTISET_EXCEPT_DISTINCT.method, NullPolicy.NONE);
+ defineMethod(MULTISET_EXCEPT, BuiltInMethod.MULTISET_EXCEPT_ALL.method, NullPolicy.NONE);
+ defineMethod(MULTISET_UNION_DISTINCT,
+ BuiltInMethod.MULTISET_UNION_DISTINCT.method, NullPolicy.NONE);
+ defineMethod(MULTISET_UNION, BuiltInMethod.MULTISET_UNION_ALL.method, NullPolicy.NONE);
+ final MethodImplementor subMultisetImplementor =
+ new MethodImplementor(BuiltInMethod.SUBMULTISET_OF.method, NullPolicy.NONE, false);
+ map.put(SUBMULTISET_OF, subMultisetImplementor);
+ map.put(NOT_SUBMULTISET_OF, NotImplementor.of(subMultisetImplementor));
+
+ map.put(COALESCE, new CoalesceImplementor());
+ map.put(CAST, new CastImplementor());
+ map.put(DATE, new CastImplementor());
+
+ map.put(REINTERPRET, new ReinterpretImplementor());
+
+ final RexCallImplementor value = new ValueConstructorImplementor();
+ map.put(MAP_VALUE_CONSTRUCTOR, value);
+ map.put(ARRAY_VALUE_CONSTRUCTOR, value);
+ map.put(ITEM, new ItemImplementor());
+
+ map.put(DEFAULT, new DefaultImplementor());
+
+ // Sequences
+ defineMethod(CURRENT_VALUE, BuiltInMethod.SEQUENCE_CURRENT_VALUE.method,
+ NullPolicy.STRICT);
+ defineMethod(NEXT_VALUE, BuiltInMethod.SEQUENCE_NEXT_VALUE.method,
+ NullPolicy.STRICT);
+
+ // Compression Operators
+ defineMethod(COMPRESS, BuiltInMethod.COMPRESS.method, NullPolicy.ARG0);
+
+ // Xml Operators
+ defineMethod(EXTRACT_VALUE, BuiltInMethod.EXTRACT_VALUE.method, NullPolicy.ARG0);
+ defineMethod(XML_TRANSFORM, BuiltInMethod.XML_TRANSFORM.method, NullPolicy.ARG0);
+ defineMethod(EXTRACT_XML, BuiltInMethod.EXTRACT_XML.method, NullPolicy.ARG0);
+ defineMethod(EXISTS_NODE, BuiltInMethod.EXISTS_NODE.method, NullPolicy.ARG0);
+
+ // Json Operators
+ defineMethod(JSON_VALUE_EXPRESSION,
+ BuiltInMethod.JSON_VALUE_EXPRESSION.method, NullPolicy.STRICT);
+ defineMethod(JSON_EXISTS, BuiltInMethod.JSON_EXISTS.method, NullPolicy.ARG0);
+ map.put(JSON_VALUE,
+ new JsonValueImplementor(BuiltInMethod.JSON_VALUE.method));
+ defineMethod(JSON_QUERY, BuiltInMethod.JSON_QUERY.method, NullPolicy.ARG0);
+ defineMethod(JSON_TYPE, BuiltInMethod.JSON_TYPE.method, NullPolicy.ARG0);
+ defineMethod(JSON_DEPTH, BuiltInMethod.JSON_DEPTH.method, NullPolicy.ARG0);
+ defineMethod(JSON_KEYS, BuiltInMethod.JSON_KEYS.method, NullPolicy.ARG0);
+ defineMethod(JSON_PRETTY, BuiltInMethod.JSON_PRETTY.method, NullPolicy.ARG0);
+ defineMethod(JSON_LENGTH, BuiltInMethod.JSON_LENGTH.method, NullPolicy.ARG0);
+ defineMethod(JSON_REMOVE, BuiltInMethod.JSON_REMOVE.method, NullPolicy.ARG0);
+ defineMethod(JSON_STORAGE_SIZE, BuiltInMethod.JSON_STORAGE_SIZE.method, NullPolicy.ARG0);
+ defineMethod(JSON_OBJECT, BuiltInMethod.JSON_OBJECT.method, NullPolicy.NONE);
+ defineMethod(JSON_ARRAY, BuiltInMethod.JSON_ARRAY.method, NullPolicy.NONE);
+ map.put(IS_JSON_VALUE,
+ new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method,
+ NullPolicy.NONE, false));
+ map.put(IS_JSON_OBJECT,
+ new MethodImplementor(BuiltInMethod.IS_JSON_OBJECT.method,
+ NullPolicy.NONE, false));
+ map.put(IS_JSON_ARRAY,
+ new MethodImplementor(BuiltInMethod.IS_JSON_ARRAY.method,
+ NullPolicy.NONE, false));
+ map.put(IS_JSON_SCALAR,
+ new MethodImplementor(BuiltInMethod.IS_JSON_SCALAR.method,
+ NullPolicy.NONE, false));
+ map.put(IS_NOT_JSON_VALUE,
+ NotImplementor.of(
+ new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method,
+ NullPolicy.NONE, false)));
+ map.put(IS_NOT_JSON_OBJECT,
+ NotImplementor.of(
+ new MethodImplementor(BuiltInMethod.IS_JSON_OBJECT.method,
+ NullPolicy.NONE, false)));
+ map.put(IS_NOT_JSON_ARRAY,
+ NotImplementor.of(
+ new MethodImplementor(BuiltInMethod.IS_JSON_ARRAY.method,
+ NullPolicy.NONE, false)));
+ map.put(IS_NOT_JSON_SCALAR,
+ NotImplementor.of(
+ new MethodImplementor(BuiltInMethod.IS_JSON_SCALAR.method,
+ NullPolicy.NONE, false)));
+
+ // System functions
+ final SystemFunctionImplementor systemFunctionImplementor = new SystemFunctionImplementor();
+ map.put(USER, systemFunctionImplementor);
+ map.put(CURRENT_USER, systemFunctionImplementor);
+ map.put(SESSION_USER, systemFunctionImplementor);
+ map.put(SYSTEM_USER, systemFunctionImplementor);
+ map.put(CURRENT_PATH, systemFunctionImplementor);
+ map.put(CURRENT_ROLE, systemFunctionImplementor);
+ map.put(CURRENT_CATALOG, systemFunctionImplementor);
+ map.put(SYSTEM_RANGE, systemFunctionImplementor);
+
+ // Current time functions
+ map.put(CURRENT_TIME, systemFunctionImplementor);
+ map.put(CURRENT_TIMESTAMP, systemFunctionImplementor);
+ map.put(CURRENT_DATE, systemFunctionImplementor);
+ map.put(LOCALTIME, systemFunctionImplementor);
+ map.put(LOCALTIMESTAMP, systemFunctionImplementor);
+ }
+
+ /** */
+ private <T> Supplier<T> constructorSupplier(Class<T> klass) {
+ final Constructor<T> constructor;
+ try {
+ constructor = klass.getDeclaredConstructor();
+ }
+ catch (NoSuchMethodException e) {
+ throw new IllegalArgumentException(
+ klass + " should implement zero arguments constructor");
+ }
+ return () -> {
+ try {
+ return constructor.newInstance();
+ }
+ catch (InstantiationException | IllegalAccessException
+ | InvocationTargetException e) {
+ throw new IllegalStateException(
+ "Error while creating aggregate implementor " + constructor, e);
+ }
+ };
+ }
+
+ /** */
+ private void defineMethod(SqlOperator operator, String functionName, NullPolicy nullPolicy) {
+ map.put(operator, new MethodNameImplementor(functionName, nullPolicy, false));
+ }
+
+ /** */
+ private void defineMethod(SqlOperator operator, Method method, NullPolicy nullPolicy) {
+ map.put(operator, new MethodImplementor(method, nullPolicy, false));
+ }
+
+ /** */
+ private void defineUnary(SqlOperator operator, ExpressionType expressionType,
+ NullPolicy nullPolicy, String backupMethodName) {
+ map.put(operator, new UnaryImplementor(expressionType, nullPolicy, backupMethodName));
+ }
+
+ /** */
+ private void defineBinary(SqlOperator operator, ExpressionType expressionType,
+ NullPolicy nullPolicy, String backupMethodName) {
+ map.put(operator,
+ new BinaryImplementor(nullPolicy, true, expressionType,
+ backupMethodName));
+ }
+
+ /** */
+ private static RexCallImplementor wrapAsRexCallImplementor(
+ final CallImplementor implementor) {
+ return new AbstractRexCallImplementor(NullPolicy.NONE, false) {
+ @Override String getVariableName() {
+ return "udf";
+ }
+
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList) {
+ return implementor.implement(translator, call, RexImpTable.NullAs.NULL);
+ }
+ };
+ }
+
+ /** */
+ public RexCallImplementor get(final SqlOperator operator) {
+ if (operator instanceof SqlUserDefinedFunction) {
+ org.apache.calcite.schema.Function udf =
+ ((SqlUserDefinedFunction)operator).getFunction();
+ if (!(udf instanceof ImplementableFunction)) {
+ throw new IllegalStateException("User defined function " + operator
+ + " must implement ImplementableFunction");
+ }
+ CallImplementor implementor = ((ImplementableFunction)udf).getImplementor();
+ return wrapAsRexCallImplementor(implementor);
+ }
+ else if (operator instanceof SqlTypeConstructorFunction)
+ return map.get(SqlStdOperatorTable.ROW);
+
+ return map.get(operator);
+ }
+
+ /** */
+ static Expression optimize(Expression expression) {
+ return expression.accept(new OptimizeShuttle());
+ }
+
+ /** */
+ static Expression optimize2(Expression operand, Expression expression) {
+ if (Primitive.is(operand.getType())) {
+ // Primitive values cannot be null
+ return optimize(expression);
+ }
+
+ return optimize(
+ Expressions.condition(
+ Expressions.equal(operand, NULL_EXPR),
+ NULL_EXPR,
+ expression));
+ }
+
+ /** */
+ private static RelDataType toSql(RelDataTypeFactory typeFactory,
+ RelDataType type) {
+ if (type instanceof RelDataTypeFactoryImpl.JavaType) {
+ final SqlTypeName typeName = type.getSqlTypeName();
+ if (typeName != null && typeName != SqlTypeName.OTHER) {
+ return typeFactory.createTypeWithNullability(
+ typeFactory.createSqlType(typeName),
+ type.isNullable());
+ }
+ }
+ return type;
+ }
+
+ /** */
+ private static <E> boolean allSame(List<E> list) {
+ E prev = null;
+ for (E e : list) {
+ if (prev != null && !prev.equals(e))
+ return false;
+
+ prev = e;
+ }
+ return true;
+ }
+
+ /**
+ * Strategy what an operator should return if one of its arguments is null.
+ */
+ public enum NullAs {
+ /**
+ * The most common policy among the SQL built-in operators. If one of the arguments is null, returns null.
+ */
+ NULL,
+
+ /**
+ * If one of the arguments is null, the function returns false. Example: {@code IS NOT NULL}.
+ */
+ FALSE,
+
+ /**
+ * If one of the arguments is null, the function returns true. Example: {@code IS NULL}.
+ */
+ TRUE,
+
+ /**
+ * It is not possible for any of the arguments to be null. If the argument type is nullable, the enclosing code
+ * will already have performed a not-null check. This may allow the operator implementor to generate a more
+ * efficient implementation, for example, by avoiding boxing or unboxing.
+ */
+ NOT_POSSIBLE,
+
+ /** Return false if result is not null, true if result is null. */
+ IS_NULL,
+
+ /** Return true if result is not null, false if result is null. */
+ IS_NOT_NULL;
+
+ public static NullAs of(boolean nullable) {
+ return nullable ? NULL : NOT_POSSIBLE;
+ }
+
+ /**
+ * Adapts an expression with "normal" result to one that adheres to this particular policy.
+ */
+ public Expression handle(Expression x) {
+ switch (Primitive.flavor(x.getType())) {
+ case PRIMITIVE:
+ // Expression cannot be null. We can skip any runtime checks.
+ switch (this) {
+ case NULL:
+ case NOT_POSSIBLE:
+ case FALSE:
+ case TRUE:
+ return x;
+ case IS_NULL:
+ return FALSE_EXPR;
+ case IS_NOT_NULL:
+ return TRUE_EXPR;
+ default:
+ throw new AssertionError();
+ }
+ case BOX:
+ switch (this) {
+ case NOT_POSSIBLE:
+ return ConverterUtils.convert(x,
+ Primitive.ofBox(x.getType()).primitiveClass);
+ }
+ // fall through
+ }
+ switch (this) {
+ case NULL:
+ case NOT_POSSIBLE:
+ return x;
+ case FALSE:
+ return Expressions.call(BuiltInMethod.IS_TRUE.method, x);
+ case TRUE:
+ return Expressions.call(BuiltInMethod.IS_NOT_FALSE.method, x);
+ case IS_NULL:
+ return Expressions.equal(x, NULL_EXPR);
+ case IS_NOT_NULL:
+ return Expressions.notEqual(x, NULL_EXPR);
+ default:
+ throw new AssertionError();
+ }
+ }
+ }
+
+ /** */
+ static Expression getDefaultValue(Type type) {
+ if (Primitive.is(type)) {
+ Primitive p = Primitive.of(type);
+ return Expressions.constant(p.defaultValue, type);
+ }
+ return Expressions.constant(null, type);
+ }
+
+ /**
+ * Multiplies an expression by a constant and divides by another constant, optimizing appropriately.
+ *
+ * <p>For example, {@code multiplyDivide(e, 10, 1000)} returns
+ * {@code e / 100}.
+ */
+ public static Expression multiplyDivide(Expression e, BigDecimal multiplier,
+ BigDecimal divider) {
+ if (multiplier.equals(BigDecimal.ONE)) {
+ if (divider.equals(BigDecimal.ONE))
+ return e;
+
+ return Expressions.divide(e,
+ Expressions.constant(divider.intValueExact()));
+ }
+ final BigDecimal x =
+ multiplier.divide(divider, RoundingMode.UNNECESSARY);
+ switch (x.compareTo(BigDecimal.ONE)) {
+ case 0:
+ return e;
+ case 1:
+ return Expressions.multiply(e, Expressions.constant(x.intValueExact()));
+ case -1:
+ return multiplyDivide(e, BigDecimal.ONE, x);
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ /** Implementor for the {@code TRIM} function. */
+ private static class TrimImplementor extends AbstractRexCallImplementor {
+ /** */
+ TrimImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "trim";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ final boolean strict = !translator.conformance.allowExtendedTrim();
+ final Object value = translator.getLiteralValue(argValueList.get(0));
+ SqlTrimFunction.Flag flag = (SqlTrimFunction.Flag)value;
+ return Expressions.call(
+ BuiltInMethod.TRIM.method,
+ Expressions.constant(
+ flag == SqlTrimFunction.Flag.BOTH
+ || flag == SqlTrimFunction.Flag.LEADING),
+ Expressions.constant(
+ flag == SqlTrimFunction.Flag.BOTH
+ || flag == SqlTrimFunction.Flag.TRAILING),
+ argValueList.get(1),
+ argValueList.get(2),
+ Expressions.constant(strict));
+ }
+ }
+
+ /**
+ * Implementor for the {@code MONTHNAME} and {@code DAYNAME} functions. Each takes a {@link java.util.Locale}
+ * argument.
+ */
+ private static class PeriodNameImplementor extends MethodNameImplementor {
+ /** */
+ private final BuiltInMethod timestampMethod;
+
+ /** */
+ private final BuiltInMethod dateMethod;
+
+ /** */
+ PeriodNameImplementor(String methodName, BuiltInMethod timestampMethod,
+ BuiltInMethod dateMethod) {
+ super(methodName, NullPolicy.STRICT, false);
+ this.timestampMethod = timestampMethod;
+ this.dateMethod = dateMethod;
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "periodName";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ Expression operand = argValueList.get(0);
+ final RelDataType type = call.operands.get(0).getType();
+ switch (type.getSqlTypeName()) {
+ case TIMESTAMP:
+ return getExpression(translator, operand, timestampMethod);
+ case DATE:
+ return getExpression(translator, operand, dateMethod);
+ default:
+ throw new AssertionError("unknown type " + type);
+ }
+ }
+
+ /** */
+ protected Expression getExpression(RexToLixTranslator translator,
+ Expression operand, BuiltInMethod builtInMethod) {
+ final MethodCallExpression locale =
+ Expressions.call(BuiltInMethod.LOCALE.method, translator.getRoot());
+ return Expressions.call(builtInMethod.method.getDeclaringClass(),
+ builtInMethod.method.getName(), operand, locale);
+ }
+ }
+
+ /** Implementor for the {@code FLOOR} and {@code CEIL} functions. */
+ private static class FloorImplementor extends MethodNameImplementor {
+ /** */
+ final Method timestampMethod;
+
+ /** */
+ final Method dateMethod;
+
+ /** */
+ FloorImplementor(String methodName, Method timestampMethod, Method dateMethod) {
+ super(methodName, NullPolicy.STRICT, false);
+ this.timestampMethod = timestampMethod;
+ this.dateMethod = dateMethod;
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "floor";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ switch (call.getOperands().size()) {
+ case 1:
+ switch (call.getType().getSqlTypeName()) {
+ case BIGINT:
+ case INTEGER:
+ case SMALLINT:
+ case TINYINT:
+ return argValueList.get(0);
+ default:
+ return super.implementSafe(translator, call, argValueList);
+ }
+
+ case 2:
+ final Type type;
+ final Method floorMethod;
+ final boolean preFloor;
+ Expression operand = argValueList.get(0);
+ switch (call.getType().getSqlTypeName()) {
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ operand = Expressions.call(
+ BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+ // fall through
+ case TIMESTAMP:
+ type = long.class;
+ floorMethod = timestampMethod;
+ preFloor = true;
+ break;
+ default:
+ type = int.class;
+ floorMethod = dateMethod;
+ preFloor = false;
+ }
+ final TimeUnitRange timeUnitRange =
+ (TimeUnitRange)translator.getLiteralValue(argValueList.get(1));
+ switch (timeUnitRange) {
+ case YEAR:
+ case QUARTER:
+ case MONTH:
+ case WEEK:
+ case DAY:
+ final Expression operand1 =
+ preFloor ? call(operand, type, TimeUnit.DAY) : operand;
+ return Expressions.call(floorMethod,
+ translator.getLiteral(argValueList.get(1)), operand1);
+ case NANOSECOND:
+ default:
+ return call(operand, type, timeUnitRange.startUnit);
+ }
+
+ default:
+ throw new AssertionError();
+ }
+ }
+
+ /** */
+ private Expression call(Expression operand, Type type,
+ TimeUnit timeUnit) {
+ return Expressions.call(SqlFunctions.class, methodName,
+ ConverterUtils.convert(operand, type),
+ ConverterUtils.convert(
+ Expressions.constant(timeUnit.multiplier), type));
+ }
+ }
+
+ /** Implementor for a function that generates calls to a given method. */
+ private static class MethodImplementor extends AbstractRexCallImplementor {
+ /** */
+ protected final Method method;
+
+ /** */
+ MethodImplementor(Method method, NullPolicy nullPolicy, boolean harmonize) {
+ super(nullPolicy, harmonize);
+ this.method = method;
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "method_call";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList) {
+ final Expression expression;
+ Class clazz = method.getDeclaringClass();
+ if (Modifier.isStatic(method.getModifiers()))
+ expression = EnumUtils.call(clazz, method.getName(), argValueList);
+ else {
+ expression = EnumUtils.call(clazz, method.getName(),
+ Util.skip(argValueList, 1), argValueList.get(0));
+ }
+ return expression;
+ }
+ }
+
+ /**
+ * Implementor for JSON_VALUE function, convert to solid format "JSON_VALUE(json_doc, path, empty_behavior,
+ * empty_default, error_behavior, error default)" in order to simplify the runtime implementation.
+ *
+ * <p>We should avoid this when we support
+ * variable arguments function.
+ */
+ private static class JsonValueImplementor extends MethodImplementor {
+ /** */
+ JsonValueImplementor(Method method) {
+ super(method, NullPolicy.ARG0, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList) {
+ final List<Expression> newOperands = new ArrayList<>();
+ newOperands.add(argValueList.get(0));
+ newOperands.add(argValueList.get(1));
+ List<Expression> leftExprs = Util.skip(argValueList, 2);
+ // Default value for JSON_VALUE behaviors.
+ Expression emptyBehavior = Expressions.constant(SqlJsonValueEmptyOrErrorBehavior.NULL);
+ Expression defaultValueOnEmpty = Expressions.constant(null);
+ Expression errorBehavior = Expressions.constant(SqlJsonValueEmptyOrErrorBehavior.NULL);
+ Expression defaultValueOnError = Expressions.constant(null);
+ // Patched up with user defines.
+ if (!leftExprs.isEmpty()) {
+ for (int i = 0; i < leftExprs.size(); i++) {
+ Expression expr = leftExprs.get(i);
+ final Object exprVal = translator.getLiteralValue(expr);
+ if (exprVal != null) {
+ int defaultSymbolIdx = i - 2;
+ if (exprVal == SqlJsonEmptyOrError.EMPTY) {
+ if (defaultSymbolIdx >= 0
+ && translator.getLiteralValue(leftExprs.get(defaultSymbolIdx))
+ == SqlJsonValueEmptyOrErrorBehavior.DEFAULT) {
+ defaultValueOnEmpty = leftExprs.get(i - 1);
+ emptyBehavior = leftExprs.get(defaultSymbolIdx);
+ }
+ else
+ emptyBehavior = leftExprs.get(i - 1);
+ }
+ else if (exprVal == SqlJsonEmptyOrError.ERROR) {
+ if (defaultSymbolIdx >= 0
+ && translator.getLiteralValue(leftExprs.get(defaultSymbolIdx))
+ == SqlJsonValueEmptyOrErrorBehavior.DEFAULT) {
+ defaultValueOnError = leftExprs.get(i - 1);
+ errorBehavior = leftExprs.get(defaultSymbolIdx);
+ }
+ else
+ errorBehavior = leftExprs.get(i - 1);
+ }
+ }
+ }
+ }
+ newOperands.add(emptyBehavior);
+ newOperands.add(defaultValueOnEmpty);
+ newOperands.add(errorBehavior);
+ newOperands.add(defaultValueOnError);
+ Class clazz = method.getDeclaringClass();
+ final Expression expression = EnumUtils.call(clazz, method.getName(), newOperands);
+
+ final Type returnType =
+ translator.typeFactory.getJavaClass(call.getType());
+ return EnumUtils.convert(expression, returnType);
+ }
+ }
+
+ /**
+ * Implementor for SQL functions that generates calls to a given method name.
+ *
+ * <p>Use this, as opposed to {@link MethodImplementor}, if the SQL function
+ * is overloaded; then you can use one implementor for several overloads.
+ */
+ private static class MethodNameImplementor extends AbstractRexCallImplementor {
+ /** */
+ protected final String methodName;
+
+ /** */
+ MethodNameImplementor(String methodName,
+ NullPolicy nullPolicy, boolean harmonize) {
+ super(nullPolicy, harmonize);
+ this.methodName = methodName;
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "method_name_call";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList) {
+ return EnumUtils.call(
+ SqlFunctions.class,
+ methodName,
+ argValueList);
+ }
+ }
+
+ /** Implementor for binary operators. */
+ private static class BinaryImplementor extends AbstractRexCallImplementor {
+ /**
+ * Types that can be arguments to comparison operators such as {@code <}.
+ */
+ private static final List<Primitive> COMP_OP_TYPES =
+ ImmutableList.of(
+ Primitive.BYTE,
+ Primitive.CHAR,
+ Primitive.SHORT,
+ Primitive.INT,
+ Primitive.LONG,
+ Primitive.FLOAT,
+ Primitive.DOUBLE);
+
+ /** */
+ private static final List<SqlBinaryOperator> COMPARISON_OPERATORS =
+ ImmutableList.of(
+ SqlStdOperatorTable.LESS_THAN,
+ SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+ SqlStdOperatorTable.GREATER_THAN,
+ SqlStdOperatorTable.GREATER_THAN_OR_EQUAL);
+
+ /** */
+ private static final List<SqlBinaryOperator> EQUALS_OPERATORS =
+ ImmutableList.of(
+ SqlStdOperatorTable.EQUALS,
+ SqlStdOperatorTable.NOT_EQUALS);
+
+ /** */
+ public static final String METHOD_POSTFIX_FOR_ANY_TYPE = "Any";
+
+ /** */
+ private final ExpressionType expressionType;
+
+ /** */
+ private final String backupMethodName;
+
+ /** */
+ BinaryImplementor(NullPolicy nullPolicy, boolean harmonize,
+ ExpressionType expressionType, String backupMethodName) {
+ super(nullPolicy, harmonize);
+ this.expressionType = expressionType;
+ this.backupMethodName = backupMethodName;
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "binary_call";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(
+ final RexToLixTranslator translator,
+ final RexCall call,
+ final List<Expression> argValueList) {
+ // neither nullable:
+ // return x OP y
+ // x nullable
+ // null_returns_null
+ // return x == null ? null : x OP y
+ // ignore_null
+ // return x == null ? null : y
+ // x, y both nullable
+ // null_returns_null
+ // return x == null || y == null ? null : x OP y
+ // ignore_null
+ // return x == null ? y : y == null ? x : x OP y
+ if (backupMethodName != null) {
+ // If one or both operands have ANY type, use the late-binding backup
+ // method.
+ if (anyAnyOperands(call))
+ return callBackupMethodAnyType(argValueList);
+
+ final Type type0 = argValueList.get(0).getType();
+ final Type type1 = argValueList.get(1).getType();
+ final SqlBinaryOperator op = (SqlBinaryOperator)call.getOperator();
+ final RelDataType relDataType0 = call.getOperands().get(0).getType();
+ final Expression fieldComparator = generateCollatorExpression(relDataType0.getCollation());
+ if (fieldComparator != null)
+ argValueList.add(fieldComparator);
+
+ final Primitive primitive = Primitive.ofBoxOr(type0);
+ if (primitive == null
+ || type1 == BigDecimal.class
+ || COMPARISON_OPERATORS.contains(op)
+ && !COMP_OP_TYPES.contains(primitive)) {
+ return Expressions.call(SqlFunctions.class, backupMethodName,
+ argValueList);
+ }
+ // When checking equals or not equals on two primitive boxing classes
+ // (i.e. Long x, Long y), we should fall back to call `SqlFunctions.eq(x, y)`
+ // or `SqlFunctions.ne(x, y)`, rather than `x == y`
+ final Primitive boxPrimitive0 = Primitive.ofBox(type0);
+ final Primitive boxPrimitive1 = Primitive.ofBox(type1);
+ if (EQUALS_OPERATORS.contains(op)
+ && boxPrimitive0 != null && boxPrimitive1 != null) {
+ return Expressions.call(SqlFunctions.class, backupMethodName,
+ argValueList);
+ }
+ }
+ return Expressions.makeBinary(expressionType,
+ argValueList.get(0), argValueList.get(1));
+ }
+
+ /** Returns whether any of a call's operands have ANY type. */
+ private boolean anyAnyOperands(RexCall call) {
+ for (RexNode operand : call.operands) {
+ if (operand.getType().getSqlTypeName() == SqlTypeName.ANY)
+ return true;
+ }
+ return false;
+ }
+
+ /** */
+ private Expression callBackupMethodAnyType(List<Expression> expressions) {
+ final String backupMethodNameForAnyType =
+ backupMethodName + METHOD_POSTFIX_FOR_ANY_TYPE;
+
+ // one or both of parameter(s) is(are) ANY type
+ final Expression expression0 = maybeBox(expressions.get(0));
+ final Expression expression1 = maybeBox(expressions.get(1));
+ return Expressions.call(SqlFunctions.class, backupMethodNameForAnyType,
+ expression0, expression1);
+ }
+
+ /** */
+ private Expression maybeBox(Expression expression) {
+ final Primitive primitive = Primitive.of(expression.getType());
+ if (primitive != null)
+ expression = Expressions.box(expression, primitive);
+
+ return expression;
+ }
+ }
+
+ /** Implementor for unary operators. */
+ private static class UnaryImplementor extends AbstractRexCallImplementor {
+ /** */
+ private final ExpressionType expressionType;
+
+ /** */
+ private final String backupMethodName;
+
+ /** */
+ UnaryImplementor(ExpressionType expressionType, NullPolicy nullPolicy,
+ String backupMethodName) {
+ super(nullPolicy, false);
+ this.expressionType = expressionType;
+ this.backupMethodName = backupMethodName;
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "unary_call";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList) {
+ final Expression argValue = argValueList.get(0);
+
+ final Expression e;
+ //Special case for implementing unary minus with BigDecimal type
+ //for other data type(except BigDecimal) '-' operator is OK, but for
+ //BigDecimal, we should call negate method of BigDecimal
+ if (expressionType == ExpressionType.Negate && argValue.type == BigDecimal.class
+ && null != backupMethodName)
+ e = Expressions.call(argValue, backupMethodName);
+ else
+ e = Expressions.makeUnary(expressionType, argValue);
+
+ if (e.type.equals(argValue.type))
+ return e;
+ // Certain unary operators do not preserve type. For example, the "-"
+ // operator applied to a "byte" expression returns an "int".
+ return Expressions.convert_(e, argValue.type);
+ }
+ }
+
+ /** Implementor for the {@code EXTRACT(unit FROM datetime)} function. */
+ private static class ExtractImplementor extends AbstractRexCallImplementor {
+ /** */
+ ExtractImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "extract";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ final TimeUnitRange timeUnitRange =
+ (TimeUnitRange)translator.getLiteralValue(argValueList.get(0));
+ final TimeUnit unit = timeUnitRange.startUnit;
+ Expression operand = argValueList.get(1);
+ final SqlTypeName sqlTypeName =
+ call.operands.get(1).getType().getSqlTypeName();
+ switch (unit) {
+ case MILLENNIUM:
+ case CENTURY:
+ case YEAR:
+ case QUARTER:
+ case MONTH:
+ case DAY:
+ case DOW:
+ case DECADE:
+ case DOY:
+ case ISODOW:
+ case ISOYEAR:
+ case WEEK:
+ switch (sqlTypeName) {
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ break;
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ operand = Expressions.call(
+ BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+ // fall through
+ case TIMESTAMP:
+ operand = Expressions.call(BuiltInMethod.FLOOR_DIV.method,
+ operand, Expressions.constant(TimeUnit.DAY.multiplier.longValue()));
+ // fall through
+ case DATE:
+ return Expressions.call(BuiltInMethod.UNIX_DATE_EXTRACT.method,
+ argValueList.get(0), operand);
+ default:
+ throw new AssertionError("unexpected " + sqlTypeName);
+ }
+ break;
+ case MILLISECOND:
+ case MICROSECOND:
+ case NANOSECOND:
+ if (sqlTypeName == SqlTypeName.DATE)
+ return Expressions.constant(0L);
+
+ operand = mod(operand, TimeUnit.MINUTE.multiplier.longValue());
+ return Expressions.multiply(
+ operand, Expressions.constant((long)(1 / unit.multiplier.doubleValue())));
+ case EPOCH:
+ switch (sqlTypeName) {
+ case DATE:
+ // convert to milliseconds
+ operand = Expressions.multiply(operand,
+ Expressions.constant(TimeUnit.DAY.multiplier.longValue()));
+ // fall through
+ case TIMESTAMP:
+ // convert to seconds
+ return Expressions.divide(operand,
+ Expressions.constant(TimeUnit.SECOND.multiplier.longValue()));
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ operand = Expressions.call(
+ BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
+ return Expressions.divide(operand,
+ Expressions.constant(TimeUnit.SECOND.multiplier.longValue()));
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ // no convertlet conversion, pass it as extract
+ throw new AssertionError("unexpected " + sqlTypeName);
+ }
+ break;
+ case HOUR:
+ case MINUTE:
+ case SECOND:
+ switch (sqlTypeName) {
+ case DATE:
+ return Expressions.multiply(operand, Expressions.constant(0L));
+ }
+ break;
+ }
+
+ operand = mod(operand, getFactor(unit));
+ if (unit == TimeUnit.QUARTER)
+ operand = Expressions.subtract(operand, Expressions.constant(1L));
+
+ operand = Expressions.divide(operand,
+ Expressions.constant(unit.multiplier.longValue()));
+ if (unit == TimeUnit.QUARTER)
+ operand = Expressions.add(operand, Expressions.constant(1L));
+
+ return operand;
+ }
+ }
+
+ /** */
+ private static Expression mod(Expression operand, long factor) {
+ if (factor == 1L)
+ return operand;
+ else {
+ return Expressions.call(BuiltInMethod.FLOOR_MOD.method,
+ operand, Expressions.constant(factor));
+ }
+ }
+
+ /** */
+ private static long getFactor(TimeUnit unit) {
+ switch (unit) {
+ case DAY:
+ return 1L;
+ case HOUR:
+ return TimeUnit.DAY.multiplier.longValue();
+ case MINUTE:
+ return TimeUnit.HOUR.multiplier.longValue();
+ case SECOND:
+ return TimeUnit.MINUTE.multiplier.longValue();
+ case MILLISECOND:
+ return TimeUnit.SECOND.multiplier.longValue();
+ case MONTH:
+ return TimeUnit.YEAR.multiplier.longValue();
+ case QUARTER:
+ return TimeUnit.YEAR.multiplier.longValue();
+ case YEAR:
+ case DECADE:
+ case CENTURY:
+ case MILLENNIUM:
+ return 1L;
+ default:
+ throw Util.unexpected(unit);
+ }
+ }
+
+ /** Implementor for the SQL {@code COALESCE} operator. */
+ private static class CoalesceImplementor extends AbstractRexCallImplementor {
+ /** */
+ CoalesceImplementor() {
+ super(NullPolicy.NONE, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "coalesce";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return implementRecurse(translator, argValueList);
+ }
+
+ /** */
+ private Expression implementRecurse(RexToLixTranslator translator,
+ final List<Expression> argValueList) {
+ if (argValueList.size() == 1)
+ return argValueList.get(0);
+ else {
+ return Expressions.condition(
+ translator.checkNotNull(argValueList.get(0)),
+ argValueList.get(0),
+ implementRecurse(translator, Util.skip(argValueList)));
+ }
+ }
+ }
+
+ /** Implementor for the SQL {@code CAST} operator. */
+ private static class CastImplementor extends AbstractRexCallImplementor {
+ /** */
+ CastImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "cast";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ assert call.getOperands().size() == 1;
+ final RelDataType sourceType = call.getOperands().get(0).getType();
+
+ // Short-circuit if no cast is required
+ RexNode arg = call.getOperands().get(0);
+ if (call.getType().equals(sourceType)) {
+ // No cast required, omit cast
+ return argValueList.get(0);
+ }
+ if (SqlTypeUtil.equalSansNullability(translator.typeFactory,
+ call.getType(), arg.getType())
+ && translator.deref(arg) instanceof RexLiteral) {
+ return RexToLixTranslator.translateLiteral(
+ (RexLiteral)translator.deref(arg), call.getType(),
+ translator.typeFactory, NullAs.NULL);
+ }
+ final RelDataType targetType =
+ nullifyType(translator.typeFactory, call.getType(), false);
+ return translator.translateCast(sourceType,
+ targetType, argValueList.get(0));
+ }
+
+ /** */
+ private RelDataType nullifyType(JavaTypeFactory typeFactory,
+ final RelDataType type, final boolean nullable) {
+ if (type instanceof RelDataTypeFactoryImpl.JavaType) {
+ final Primitive primitive = Primitive.ofBox(
+ ((RelDataTypeFactoryImpl.JavaType)type).getJavaClass());
+ if (primitive != null)
+ return typeFactory.createJavaType(primitive.primitiveClass);
+ }
+ return typeFactory.createTypeWithNullability(type, nullable);
+ }
+ }
+
+ /** Implementor for the {@code REINTERPRET} internal SQL operator. */
+ private static class ReinterpretImplementor extends AbstractRexCallImplementor {
+ /** */
+ ReinterpretImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "reInterpret";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ assert call.getOperands().size() == 1;
+ return argValueList.get(0);
+ }
+ }
+
+ /** Implementor for a value-constructor. */
+ private static class ValueConstructorImplementor
+ extends AbstractRexCallImplementor {
+
+ /** */
+ ValueConstructorImplementor() {
+ super(NullPolicy.NONE, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "value_constructor";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ SqlKind kind = call.getOperator().getKind();
+ final BlockBuilder blockBuilder = translator.getBlockBuilder();
+ switch (kind) {
+ case MAP_VALUE_CONSTRUCTOR:
+ Expression map =
+ blockBuilder.append("map", Expressions.new_(LinkedHashMap.class),
+ false);
+ for (int i = 0; i < argValueList.size(); i++) {
+ Expression key = argValueList.get(i++);
+ Expression value = argValueList.get(i);
+ blockBuilder.add(
+ Expressions.statement(
+ Expressions.call(map, BuiltInMethod.MAP_PUT.method,
+ Expressions.box(key), Expressions.box(value))));
+ }
+ return map;
+ case ARRAY_VALUE_CONSTRUCTOR:
+ Expression lyst =
+ blockBuilder.append("list", Expressions.new_(ArrayList.class),
+ false);
+ for (Expression value : argValueList) {
+ blockBuilder.add(
+ Expressions.statement(
+ Expressions.call(lyst, BuiltInMethod.COLLECTION_ADD.method,
+ Expressions.box(value))));
+ }
+ return lyst;
+ default:
+ throw new AssertionError("unexpected: " + kind);
+ }
+ }
+ }
+
+ /** Implementor for the {@code ITEM} SQL operator. */
+ private static class ItemImplementor extends AbstractRexCallImplementor {
+ /** */
+ ItemImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "item";
+ }
+
+ // Since we follow PostgreSQL's semantics that an out-of-bound reference
+ // returns NULL, x[y] can return null even if x and y are both NOT NULL.
+ // (In SQL standard semantics, an out-of-bound reference to an array
+ // throws an exception.)
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ final MethodImplementor implementor =
+ getImplementor(call.getOperands().get(0).getType().getSqlTypeName());
+ return implementor.implementSafe(translator, call, argValueList);
+ }
+
+ /** */
+ private MethodImplementor getImplementor(SqlTypeName sqlTypeName) {
+ switch (sqlTypeName) {
+ case ARRAY:
+ return new MethodImplementor(BuiltInMethod.ARRAY_ITEM.method, nullPolicy, false);
+ case MAP:
+ return new MethodImplementor(BuiltInMethod.MAP_ITEM.method, nullPolicy, false);
+ default:
+ return new MethodImplementor(BuiltInMethod.ANY_ITEM.method, nullPolicy, false);
+ }
+ }
+ }
+
+ /**
+ * Implementor for SQL system functions.
+ *
+ * <p>Several of these are represented internally as constant values, set
+ * per execution.
+ */
+ private static class SystemFunctionImplementor
+ extends AbstractRexCallImplementor {
+ /** */
+ SystemFunctionImplementor() {
+ super(NullPolicy.NONE, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "system_func";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ final SqlOperator op = call.getOperator();
+ final Expression root = translator.getRoot();
+ if (op == CURRENT_USER
+ || op == SESSION_USER
+ || op == USER)
+ return Expressions.call(BuiltInMethod.USER.method, root);
+ else if (op == SYSTEM_USER)
+ return Expressions.call(BuiltInMethod.SYSTEM_USER.method, root);
+ else if (op == CURRENT_PATH
+ || op == CURRENT_ROLE
+ || op == CURRENT_CATALOG) {
+ // By default, the CURRENT_ROLE and CURRENT_CATALOG functions return the
+ // empty string because a role or a catalog has to be set explicitly.
+ return Expressions.constant("");
+ }
+ else if (op == CURRENT_TIMESTAMP)
+ return Expressions.call(BuiltInMethod.CURRENT_TIMESTAMP.method, root);
+ else if (op == CURRENT_TIME)
+ return Expressions.call(BuiltInMethod.CURRENT_TIME.method, root);
+ else if (op == CURRENT_DATE)
+ return Expressions.call(BuiltInMethod.CURRENT_DATE.method, root);
+ else if (op == LOCALTIMESTAMP)
+ return Expressions.call(BuiltInMethod.LOCAL_TIMESTAMP.method, root);
+ else if (op == LOCALTIME)
+ return Expressions.call(BuiltInMethod.LOCAL_TIME.method, root);
+ else if (op == SYSTEM_RANGE) {
+ if (call.getOperands().size() == 2)
+ return createTableFunctionImplementor(IgniteMethod.SYSTEM_RANGE2.method())
+ .implement(translator, call, NullAs.NULL);
+
+ if (call.getOperands().size() == 3)
+ return createTableFunctionImplementor(IgniteMethod.SYSTEM_RANGE3.method())
+ .implement(translator, call, NullAs.NULL);
+ }
+
+ throw new AssertionError("unknown function " + op);
+ }
+ }
+
+ /** Implementor for the {@code NOT} operator. */
+ private static class NotImplementor extends AbstractRexCallImplementor {
+ /** */
+ private AbstractRexCallImplementor implementor;
+
+ /** */
+ private NotImplementor(AbstractRexCallImplementor implementor) {
+ super(null, false);
+ this.implementor = implementor;
+ }
+
+ /** */
+ static AbstractRexCallImplementor of(AbstractRexCallImplementor implementor) {
+ return new NotImplementor(implementor);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "not";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ final Expression expression =
+ implementor.implementSafe(translator, call, argValueList);
+ return Expressions.not(expression);
+ }
+ }
+
+ /** Implementor for various datetime arithmetic. */
+ private static class DatetimeArithmeticImplementor
+ extends AbstractRexCallImplementor {
+ /** */
+ DatetimeArithmeticImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "dateTime_arithmetic";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ final RexNode operand0 = call.getOperands().get(0);
+ Expression trop0 = argValueList.get(0);
+ final SqlTypeName typeName1 =
+ call.getOperands().get(1).getType().getSqlTypeName();
+ Expression trop1 = argValueList.get(1);
+ final SqlTypeName typeName = call.getType().getSqlTypeName();
+ switch (operand0.getType().getSqlTypeName()) {
+ case DATE:
+ switch (typeName) {
+ case TIMESTAMP:
+ trop0 = Expressions.convert_(
+ Expressions.multiply(trop0,
+ Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+ long.class);
+ break;
+ default:
+ switch (typeName1) {
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ trop1 = Expressions.convert_(
+ Expressions.divide(trop1,
+ Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+ int.class);
+ }
+ }
+ break;
+ case TIME:
+ trop1 = Expressions.convert_(trop1, int.class);
+ break;
+ }
+ switch (typeName1) {
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ switch (call.getKind()) {
+ case MINUS:
+ trop1 = Expressions.negate(trop1);
+ }
+ switch (typeName) {
+ case TIME:
+ return Expressions.convert_(trop0, long.class);
+ default:
+ final BuiltInMethod method =
+ operand0.getType().getSqlTypeName() == SqlTypeName.TIMESTAMP
+ ? BuiltInMethod.ADD_MONTHS
+ : BuiltInMethod.ADD_MONTHS_INT;
+ return Expressions.call(method.method, trop0, trop1);
+ }
+
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ switch (call.getKind()) {
+ case MINUS:
+ return normalize(typeName, Expressions.subtract(trop0, trop1));
+ default:
+ return normalize(typeName, Expressions.add(trop0, trop1));
+ }
+
+ default:
+ switch (call.getKind()) {
+ case MINUS:
+ switch (typeName) {
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ return Expressions.call(BuiltInMethod.SUBTRACT_MONTHS.method,
+ trop0, trop1);
+ }
+ TimeUnit fromUnit =
+ typeName1 == SqlTypeName.DATE ? TimeUnit.DAY : TimeUnit.MILLISECOND;
+ TimeUnit toUnit = TimeUnit.MILLISECOND;
+ return multiplyDivide(
+ Expressions.convert_(Expressions.subtract(trop0, trop1),
+ (Class)long.class),
+ fromUnit.multiplier, toUnit.multiplier);
+ default:
+ throw new AssertionError(call);
+ }
+ }
+ }
+
+ /** Normalizes a TIME value into 00:00:00..23:59:39. */
+ private Expression normalize(SqlTypeName typeName, Expression e) {
+ switch (typeName) {
+ case TIME:
+ return Expressions.call(BuiltInMethod.FLOOR_MOD.method, e,
+ Expressions.constant(DateTimeUtils.MILLIS_PER_DAY));
+ default:
+ return e;
+ }
+ }
+ }
+
+ /** Null-safe implementor of {@code RexCall}s. */
+ public interface RexCallImplementor {
+ /** */
+ RexToLixTranslator.Result implement(
+ RexToLixTranslator translator,
+ RexCall call,
+ List<RexToLixTranslator.Result> arguments);
+ }
+
+ /**
+ * Abstract implementation of the {@link RexCallImplementor} interface.
+ *
+ * <p>It is not always safe to execute the {@link RexCall} directly due to
+ * the special null arguments. Therefore, the generated code logic is conditional correspondingly.
+ *
+ * <p>For example, {@code a + b} will generate two declaration statements:
+ *
+ * <blockquote>
+ * <code>
+ * final Integer xxx_value = (a_isNull || b_isNull) ? null : plus(a, b);<br> final boolean xxx_isNull = xxx_value ==
+ * null;
+ * </code>
+ * </blockquote>
+ */
+ private abstract static class AbstractRexCallImplementor
+ implements RexCallImplementor {
+ /** */
+ final NullPolicy nullPolicy;
+
+ /** */
+ private final boolean harmonize;
+
+ /** */
+ AbstractRexCallImplementor(NullPolicy nullPolicy, boolean harmonize) {
+ this.nullPolicy = nullPolicy;
+ this.harmonize = harmonize;
+ }
+
+ /** {@inheritDoc} */
+ @Override public RexToLixTranslator.Result implement(
+ final RexToLixTranslator translator,
+ final RexCall call,
+ final List<RexToLixTranslator.Result> arguments) {
+ final List<Expression> argIsNullList = new ArrayList<>();
+ final List<Expression> argValueList = new ArrayList<>();
+ for (RexToLixTranslator.Result result : arguments) {
+ argIsNullList.add(result.isNullVariable);
+ argValueList.add(result.valueVariable);
+ }
+ final Expression condition = getCondition(argIsNullList);
+ final ParameterExpression valueVariable =
+ genValueStatement(translator, call, argValueList, condition);
+ final ParameterExpression isNullVariable =
+ genIsNullStatement(translator, valueVariable);
+ return new RexToLixTranslator.Result(isNullVariable, valueVariable);
+ }
+
+ /** */
+ // Variable name facilitates reasoning about issues when necessary
+ abstract String getVariableName();
+
+ /** Figures out conditional expression according to NullPolicy. */
+ Expression getCondition(final List<Expression> argIsNullList) {
+ if (argIsNullList.isEmpty()
+ || nullPolicy == null
+ || nullPolicy == NullPolicy.NONE)
+ return FALSE_EXPR;
+
+ if (nullPolicy == NullPolicy.ARG0)
+ return argIsNullList.get(0);
+
+ return Expressions.foldOr(argIsNullList);
+ }
+
+ /** */
+ // E.g., "final Integer xxx_value = (a_isNull || b_isNull) ? null : plus(a, b)"
+ private ParameterExpression genValueStatement(
+ final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList,
+ final Expression condition) {
+ List<Expression> optimizedArgValueList = argValueList;
+ if (harmonize) {
+ optimizedArgValueList =
+ harmonize(optimizedArgValueList, translator, call);
+ }
+ optimizedArgValueList = unboxIfNecessary(optimizedArgValueList);
+
+ final Expression callValue =
+ implementSafe(translator, call, optimizedArgValueList);
+
+ // In general, RexCall's type is correct for code generation
+ // and thus we should ensure the consistency.
+ // However, for some special cases (e.g., TableFunction),
+ // the implementation's type is correct, we can't convert it.
+ final SqlOperator op = call.getOperator();
+ final Type returnType = translator.typeFactory.getJavaClass(call.getType());
+ final boolean noConvert = (returnType == null)
+ || (returnType == callValue.getType())
+ || (op instanceof SqlTableFunction)
+ || (op instanceof SqlUserDefinedTableMacro)
+ || (op instanceof SqlUserDefinedTableFunction);
+ final Expression convertedCallValue =
+ noConvert
+ ? callValue
+ : ConverterUtils.convert(callValue, returnType);
+
+ final Expression valueExpression =
+ Expressions.condition(condition,
+ getIfTrue(convertedCallValue.getType()),
+ convertedCallValue);
+ final ParameterExpression value =
+ Expressions.parameter(convertedCallValue.getType(),
+ translator.getBlockBuilder().newName(getVariableName() + "_value"));
+ translator.getBlockBuilder().add(
+ Expressions.declare(Modifier.FINAL, value, valueExpression));
+ return value;
+ }
+
+ /** */
+ Expression getIfTrue(Type type) {
+ return getDefaultValue(type);
+ }
+
+ /** */
+ // E.g., "final boolean xxx_isNull = xxx_value == null"
+ private ParameterExpression genIsNullStatement(
+ final RexToLixTranslator translator, final ParameterExpression value) {
+ final ParameterExpression isNullVariable =
+ Expressions.parameter(Boolean.TYPE,
+ translator.getBlockBuilder().newName(getVariableName() + "_isNull"));
+ final Expression isNullExpression = translator.checkNull(value);
+ translator.getBlockBuilder().add(
+ Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+ return isNullVariable;
+ }
+
+ /** Ensures that operands have identical type. */
+ private List<Expression> harmonize(final List<Expression> argValueList,
+ final RexToLixTranslator translator, final RexCall call) {
+ int nullCount = 0;
+ final List<RelDataType> types = new ArrayList<>();
+ final RelDataTypeFactory typeFactory =
+ translator.builder.getTypeFactory();
+ for (RexNode operand : call.getOperands()) {
+ RelDataType type = operand.getType();
+ type = toSql(typeFactory, type);
+ if (translator.isNullable(operand))
+ ++nullCount;
+ else
+ type = typeFactory.createTypeWithNullability(type, false);
+
+ types.add(type);
+ }
+ if (allSame(types)) {
+ // Operands have the same nullability and type. Return them
+ // unchanged.
+ return argValueList;
+ }
+ final RelDataType type = typeFactory.leastRestrictive(types);
+ if (type == null) {
+ // There is no common type. Presumably this is a binary operator with
+ // asymmetric arguments (e.g. interval / integer) which is not intended
+ // to be harmonized.
+ return argValueList;
+ }
+ assert (nullCount > 0) == type.isNullable();
+ final Type javaClass =
+ translator.typeFactory.getJavaClass(type);
+ final List<Expression> harmonizedArgValues = new ArrayList<>();
+ for (Expression argValue : argValueList) {
+ harmonizedArgValues.add(
+ EnumUtils.convert(argValue, javaClass));
+ }
+ return harmonizedArgValues;
+ }
+
+ /**
+ * Under null check, it is safe to unbox the operands before entering the implementor.
+ */
+ private List<Expression> unboxIfNecessary(final List<Expression> argValueList) {
+ List<Expression> unboxValueList = argValueList;
+ if (nullPolicy == NullPolicy.STRICT || nullPolicy == NullPolicy.ANY
+ || nullPolicy == NullPolicy.SEMI_STRICT) {
+ unboxValueList = argValueList.stream()
+ .map(this::unboxExpression)
+ .collect(Collectors.toList());
+ }
+ if (nullPolicy == NullPolicy.ARG0 && !argValueList.isEmpty()) {
+ final Expression unboxArg0 = unboxExpression(unboxValueList.get(0));
+ unboxValueList.set(0, unboxArg0);
+ }
+ return unboxValueList;
+ }
+
+ /** */
+ private Expression unboxExpression(final Expression argValue) {
+ Primitive fromBox = Primitive.ofBox(argValue.getType());
+ if (fromBox == null || fromBox == Primitive.VOID)
+ return argValue;
+
+ // Optimization: for "long x";
+ // "Long.valueOf(x)" generates "x"
+ if (argValue instanceof MethodCallExpression) {
+ MethodCallExpression mce = (MethodCallExpression)argValue;
+ if ("valueOf".equals(mce.method.getName()) && mce.expressions.size() == 1) {
+ Expression originArg = mce.expressions.get(0);
+ if (Primitive.of(originArg.type) == fromBox)
+ return originArg;
+ }
+ }
+ return NullAs.NOT_POSSIBLE.handle(argValue);
+ }
+
+ /** */
+ abstract Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList);
+ }
+
+ /**
+ * Implementor for the {@code AND} operator.
+ *
+ * <p>If any of the arguments are false, result is false;
+ * else if any arguments are null, result is null; else true.
+ */
+ private static class LogicalAndImplementor extends AbstractRexCallImplementor {
+ /** */
+ LogicalAndImplementor() {
+ super(NullPolicy.NONE, true);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "logical_and";
+ }
+
+ /** {@inheritDoc} */
+ @Override public RexToLixTranslator.Result implement(final RexToLixTranslator translator,
+ final RexCall call, final List<RexToLixTranslator.Result> arguments) {
+ final List<Expression> argIsNullList = new ArrayList<>();
+ for (RexToLixTranslator.Result result : arguments)
+ argIsNullList.add(result.isNullVariable);
+
+ final List<Expression> nullAsTrue =
+ arguments.stream()
+ .map(result ->
+ Expressions.condition(result.isNullVariable, TRUE_EXPR,
+ result.valueVariable))
+ .collect(Collectors.toList());
+ final Expression hasFalse =
+ Expressions.not(Expressions.foldAnd(nullAsTrue));
+ final Expression hasNull = Expressions.foldOr(argIsNullList);
+ final Expression callExpression =
+ Expressions.condition(hasFalse, BOXED_FALSE_EXPR,
+ Expressions.condition(hasNull, NULL_EXPR, BOXED_TRUE_EXPR));
+ final RexImpTable.NullAs nullAs = translator.isNullable(call)
+ ? RexImpTable.NullAs.NULL : RexImpTable.NullAs.NOT_POSSIBLE;
+ final Expression valueExpression = nullAs.handle(callExpression);
+ final ParameterExpression valueVariable =
+ Expressions.parameter(valueExpression.getType(),
+ translator.getBlockBuilder().newName(getVariableName() + "_value"));
+ final Expression isNullExpression = translator.checkNull(valueVariable);
+ final ParameterExpression isNullVariable =
+ Expressions.parameter(Boolean.TYPE,
+ translator.getBlockBuilder().newName(getVariableName() + "_isNull"));
+ translator.getBlockBuilder().add(
+ Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+ translator.getBlockBuilder().add(
+ Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+ return new RexToLixTranslator.Result(isNullVariable, valueVariable);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return null;
+ }
+ }
+
+ /**
+ * Implementor for the {@code OR} operator.
+ *
+ * <p>If any of the arguments are true, result is true;
+ * else if any arguments are null, result is null; else false.
+ */
+ private static class LogicalOrImplementor extends AbstractRexCallImplementor {
+ /** */
+ LogicalOrImplementor() {
+ super(NullPolicy.NONE, true);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "logical_or";
+ }
+
+ /** {@inheritDoc} */
+ @Override public RexToLixTranslator.Result implement(final RexToLixTranslator translator,
+ final RexCall call, final List<RexToLixTranslator.Result> arguments) {
+ final List<Expression> argIsNullList = new ArrayList<>();
+ for (RexToLixTranslator.Result result : arguments)
+ argIsNullList.add(result.isNullVariable);
+
+ final List<Expression> nullAsFalse =
+ arguments.stream()
+ .map(result ->
+ Expressions.condition(result.isNullVariable, FALSE_EXPR,
+ result.valueVariable))
+ .collect(Collectors.toList());
+ final Expression hasTrue = Expressions.foldOr(nullAsFalse);
+ final Expression hasNull = Expressions.foldOr(argIsNullList);
+ final Expression callExpression =
+ Expressions.condition(hasTrue, BOXED_TRUE_EXPR,
+ Expressions.condition(hasNull, NULL_EXPR, BOXED_FALSE_EXPR));
+ final RexImpTable.NullAs nullAs = translator.isNullable(call)
+ ? RexImpTable.NullAs.NULL : RexImpTable.NullAs.NOT_POSSIBLE;
+ final Expression valueExpression = nullAs.handle(callExpression);
+ final ParameterExpression valueVariable =
+ Expressions.parameter(valueExpression.getType(),
+ translator.getBlockBuilder().newName(getVariableName() + "_value"));
+ final Expression isNullExpression = translator.checkNull(valueExpression);
+ final ParameterExpression isNullVariable =
+ Expressions.parameter(Boolean.TYPE,
+ translator.getBlockBuilder().newName(getVariableName() + "_isNull"));
+ translator.getBlockBuilder().add(
+ Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+ translator.getBlockBuilder().add(
+ Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+ return new RexToLixTranslator.Result(isNullVariable, valueVariable);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return null;
+ }
+ }
+
+ /**
+ * Implementor for the {@code NOT} operator.
+ *
+ * <p>If any of the arguments are false, result is true;
+ * else if any arguments are null, result is null; else false.
+ */
+ private static class LogicalNotImplementor extends AbstractRexCallImplementor {
+ /** */
+ LogicalNotImplementor() {
+ super(NullPolicy.NONE, true);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "logical_not";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.call(BuiltInMethod.NOT.method, argValueList);
+ }
+ }
+
+ /**
+ * Implementation that calls a given {@link Method}.
+ *
+ * <p>When method is not static, a new instance of the required class is
+ * created.
+ */
+ private static class ReflectiveImplementor extends AbstractRexCallImplementor {
+ /** */
+ protected final Method method;
+
+ /** */
+ ReflectiveImplementor(Method method, NullPolicy nullPolicy) {
+ super(nullPolicy, false);
+ this.method = method;
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "reflective_" + method.getName();
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList) {
+ List<Expression> argValueList0 = ConverterUtils.fromInternal(method.getParameterTypes(), argValueList);
+ if ((method.getModifiers() & Modifier.STATIC) != 0)
+ return Expressions.call(method, argValueList0);
+
+ // The UDF class must have a public zero-args constructor.
+ // Assume that the validator checked already.
+ final Expression target = Expressions.new_(method.getDeclaringClass());
+ return Expressions.call(target, method, argValueList0);
+ }
+ }
+
+ /** Implementor for the {@code RAND} function. */
+ private static class RandImplementor extends AbstractRexCallImplementor {
+ /** */
+ private final AbstractRexCallImplementor[] implementors = {
+ new ReflectiveImplementor(BuiltInMethod.RAND.method, nullPolicy),
+ new ReflectiveImplementor(BuiltInMethod.RAND_SEED.method, nullPolicy)
+ };
+
+ /** */
+ RandImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "rand";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return implementors[call.getOperands().size()]
+ .implementSafe(translator, call, argValueList);
+ }
+ }
+
+ /** Implementor for the {@code RAND_INTEGER} function. */
+ private static class RandIntegerImplementor extends AbstractRexCallImplementor {
+ /** */
+ private final AbstractRexCallImplementor[] implementors = {
+ null,
+ new ReflectiveImplementor(BuiltInMethod.RAND_INTEGER.method, nullPolicy),
+ new ReflectiveImplementor(BuiltInMethod.RAND_INTEGER_SEED.method, nullPolicy)
+ };
+
+ /** */
+ RandIntegerImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "rand_integer";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return implementors[call.getOperands().size()]
+ .implementSafe(translator, call, argValueList);
+ }
+ }
+
+ /** Implementor for the {@code PI} operator. */
+ private static class PiImplementor extends AbstractRexCallImplementor {
+ /** */
+ PiImplementor() {
+ super(NullPolicy.NONE, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "pi";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.constant(Math.PI);
+ }
+ }
+
+ /** Implementor for the {@code IS FALSE} SQL operator. */
+ private static class IsFalseImplementor extends AbstractRexCallImplementor {
+ /** */
+ IsFalseImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "is_false";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression getIfTrue(Type type) {
+ return Expressions.constant(false, type);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.equal(argValueList.get(0), FALSE_EXPR);
+ }
+ }
+
+ /** Implementor for the {@code IS NOT FALSE} SQL operator. */
+ private static class IsNotFalseImplementor extends AbstractRexCallImplementor {
+ /** */
+ IsNotFalseImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "is_not_false";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression getIfTrue(Type type) {
+ return Expressions.constant(true, type);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.notEqual(argValueList.get(0), FALSE_EXPR);
+ }
+ }
+
+ /** Implementor for the {@code IS NOT NULL} SQL operator. */
+ private static class IsNotNullImplementor extends AbstractRexCallImplementor {
+ /** */
+ IsNotNullImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "is_not_null";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression getIfTrue(Type type) {
+ return Expressions.constant(false, type);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.notEqual(argValueList.get(0), NULL_EXPR);
+ }
+ }
+
+ /** Implementor for the {@code IS NOT TRUE} SQL operator. */
+ private static class IsNotTrueImplementor extends AbstractRexCallImplementor {
+ /** */
+ IsNotTrueImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "is_not_true";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression getIfTrue(Type type) {
+ return Expressions.constant(true, type);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.notEqual(argValueList.get(0), TRUE_EXPR);
+ }
+ }
+
+ /** Implementor for the {@code IS NULL} SQL operator. */
+ private static class IsNullImplementor extends AbstractRexCallImplementor {
+ /** */
+ IsNullImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "is_null";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression getIfTrue(Type type) {
+ return Expressions.constant(true, type);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.equal(argValueList.get(0), NULL_EXPR);
+ }
+ }
+
+ /** Implementor for the {@code IS TRUE} SQL operator. */
+ private static class IsTrueImplementor extends AbstractRexCallImplementor {
+ /** */
+ IsTrueImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "is_true";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression getIfTrue(Type type) {
+ return Expressions.constant(false, type);
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.equal(argValueList.get(0), TRUE_EXPR);
+ }
+ }
+
+ /** Implementor for the {@code REGEXP_REPLACE} function. */
+ private static class RegexpReplaceImplementor extends AbstractRexCallImplementor {
+ private final AbstractRexCallImplementor[] implementors = {
+ new ReflectiveImplementor(BuiltInMethod.REGEXP_REPLACE3.method, nullPolicy),
+ new ReflectiveImplementor(BuiltInMethod.REGEXP_REPLACE4.method, nullPolicy),
+ new ReflectiveImplementor(BuiltInMethod.REGEXP_REPLACE5.method, nullPolicy),
+ new ReflectiveImplementor(BuiltInMethod.REGEXP_REPLACE6.method, nullPolicy),
+ };
+
+ /** */
+ RegexpReplaceImplementor() {
+ super(NullPolicy.STRICT, false);
+ }
+
+ @Override String getVariableName() {
+ return "regexp_replace";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList) {
+ return implementors[call.getOperands().size() - 3]
+ .implementSafe(translator, call, argValueList);
+ }
+ }
+
+ /** Implementor for the {@code DEFAULT} function. */
+ private static class DefaultImplementor extends AbstractRexCallImplementor {
+ /** */
+ DefaultImplementor() {
+ super(NullPolicy.NONE, false);
+ }
+
+ /** {@inheritDoc} */
+ @Override String getVariableName() {
+ return "default";
+ }
+
+ /** {@inheritDoc} */
+ @Override Expression implementSafe(final RexToLixTranslator translator,
+ final RexCall call, final List<Expression> argValueList) {
+ return Expressions.constant(null);
+ }
+ }
+
+ /** */
+ private static CallImplementor createTableFunctionImplementor(final Method method) {
+ return createImplementor(
+ new ReflectiveCallNotNullImplementor(method) {
+ @Override public Expression implement(RexToLixTranslator translator,
+ RexCall call, List<Expression> translatedOperands) {
+ Expression expr = super.implement(translator, call,
+ translatedOperands);
+ final Class<?> returnType = method.getReturnType();
+ if (QueryableTable.class.isAssignableFrom(returnType)) {
+ Expression queryable = Expressions.call(
+ Expressions.convert_(expr, QueryableTable.class),
+ BuiltInMethod.QUERYABLE_TABLE_AS_QUERYABLE.method,
+ Expressions.call(translator.getRoot(),
+ BuiltInMethod.DATA_CONTEXT_GET_QUERY_PROVIDER.method),
+ Expressions.constant(null, SchemaPlus.class),
+ Expressions.constant(call.getOperator().getName(), String.class));
+ expr = Expressions.call(queryable,
+ BuiltInMethod.QUERYABLE_AS_ENUMERABLE.method);
+ } else {
+ expr = Expressions.call(expr,
+ BuiltInMethod.SCANNABLE_TABLE_SCAN.method,
+ translator.getRoot());
+ }
+ return expr;
+ }
+ }, NullPolicy.NONE, false);
+ }
+
+ /** */
+ public static CallImplementor createImplementor(
+ final NotNullImplementor implementor,
+ final NullPolicy nullPolicy,
+ final boolean harmonize) {
+ return (translator, call, nullAs) -> {
+ final RexImpTable.RexCallImplementor rexCallImplementor
+ = createRexCallImplementor(implementor, nullPolicy, harmonize);
+ final List<RexToLixTranslator.Result> arguments = translator.getCallOperandResult(call);
+ assert arguments != null;
+ final RexToLixTranslator.Result result = rexCallImplementor.implement(translator, call, arguments);
+ return nullAs.handle(result.valueVariable);
+ };
+ }
+
+ /** */
+ private static RexCallImplementor createRexCallImplementor(
+ final NotNullImplementor implementor,
+ final NullPolicy nullPolicy,
+ final boolean harmonize) {
+ return new AbstractRexCallImplementor(nullPolicy, harmonize) {
+ @Override String getVariableName() {
+ return "not_null_udf";
+ }
+
+ @Override Expression implementSafe(RexToLixTranslator translator,
+ RexCall call, List<Expression> argValueList) {
+ return implementor.implement(translator, call, argValueList);
+ }
+ };
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToLixTranslator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToLixTranslator.java
new file mode 100644
index 0000000..b980735
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/RexToLixTranslator.java
@@ -0,0 +1,1312 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp;
+
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.util.ByteString;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.BlockStatement;
+import org.apache.calcite.linq4j.tree.CatchBlock;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
+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.linq4j.tree.Primitive;
+import org.apache.calcite.linq4j.tree.Statement;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexTableInputRef;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVariable;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.runtime.GeoFunctions;
+import org.apache.calcite.runtime.Geometries;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.Pair;
+
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CASE;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SEARCH;
+
+/**
+ * Translates {@link RexNode REX expressions} to {@link Expression linq4j expressions}.
+ */
+public class RexToLixTranslator implements RexVisitor<RexToLixTranslator.Result> {
+ /** */
+ final JavaTypeFactory typeFactory;
+
+ /** */
+ final RexBuilder builder;
+
+ /** */
+ private final RexProgram program;
+
+ /** */
+ final SqlConformance conformance;
+
+ /** */
+ private final Expression root;
+
+ /** */
+ final RexToLixTranslator.InputGetter inputGetter;
+
+ /** */
+ private final BlockBuilder list;
+
+ /** */
+ private final Function1<String, InputGetter> correlates;
+
+ /**
+ * Map from RexLiteral's variable name to its literal, which is often a ({@link ConstantExpression})) It is used in
+ * the some {@code RexCall}'s implementors, such as {@code ExtractImplementor}.
+ *
+ * @see #getLiteral
+ * @see #getLiteralValue
+ */
+ private final Map<Expression, Expression> literalMap = new HashMap<>();
+
+ /**
+ * For {@code RexCall}, keep the list of its operand's {@code Result}. It is useful when creating a {@code
+ * CallImplementor}.
+ */
+ private final Map<RexCall, List<Result>> callOperandResultMap = new HashMap<>();
+
+ /**
+ * Map from RexNode under specific storage type to its Result, to avoid generating duplicate code. For {@code
+ * RexInputRef}, {@code RexDynamicParam} and {@code RexFieldAccess}.
+ */
+ private final Map<Pair<RexNode, Type>, Result> rexWithStorageTypeResultMap = new HashMap<>();
+
+ /**
+ * Map from RexNode to its Result, to avoid generating duplicate code. For {@code RexLiteral} and {@code RexCall}.
+ */
+ private final Map<RexNode, Result> rexResultMap = new HashMap<>();
+
+ /** */
+ private Type currentStorageType;
+
+ /** */
+ private RexToLixTranslator(RexProgram program,
+ JavaTypeFactory typeFactory,
+ Expression root,
+ InputGetter inputGetter,
+ BlockBuilder list,
+ RexBuilder builder,
+ SqlConformance conformance,
+ Function1<String, InputGetter> correlates) {
+ this.program = program; // may be null
+ this.typeFactory = Objects.requireNonNull(typeFactory);
+ this.conformance = Objects.requireNonNull(conformance);
+ this.root = Objects.requireNonNull(root);
+ this.inputGetter = inputGetter;
+ this.list = Objects.requireNonNull(list);
+ this.builder = Objects.requireNonNull(builder);
+ this.correlates = correlates; // may be null
+ }
+
+ /**
+ * Translates a {@link RexProgram} to a sequence of expressions and declarations.
+ *
+ * @param program Program to be translated
+ * @param typeFactory Type factory
+ * @param conformance SQL conformance
+ * @param list List of statements, populated with declarations
+ * @param outputPhysType Output type, or null
+ * @param root Root expression
+ * @param inputGetter Generates expressions for inputs
+ * @param correlates Provider of references to the values of correlated variables
+ * @return Sequence of expressions, optional condition
+ */
+ public static List<Expression> translateProjects(RexProgram program,
+ JavaTypeFactory typeFactory, SqlConformance conformance,
+ BlockBuilder list, PhysType outputPhysType, Expression root,
+ InputGetter inputGetter, Function1<String, InputGetter> correlates) {
+ List<Type> storageTypes = null;
+ if (outputPhysType != null) {
+ final RelDataType rowType = outputPhysType.getRowType();
+ storageTypes = new ArrayList<>(rowType.getFieldCount());
+ for (int i = 0; i < rowType.getFieldCount(); i++)
+ storageTypes.add(outputPhysType.getJavaFieldType(i));
+ }
+ return new RexToLixTranslator(program, typeFactory, root, inputGetter,
+ list, new RexBuilder(typeFactory), conformance, null)
+ .setCorrelates(correlates)
+ .translateList(program.getProjectList(), storageTypes);
+ }
+
+ /** */
+ Expression translate(RexNode expr) {
+ final RexImpTable.NullAs nullAs =
+ RexImpTable.NullAs.of(isNullable(expr));
+ return translate(expr, nullAs);
+ }
+
+ /** */
+ Expression translate(RexNode expr, RexImpTable.NullAs nullAs) {
+ return translate(expr, nullAs, null);
+ }
+
+ /** */
+ Expression translate(RexNode expr, Type storageType) {
+ final RexImpTable.NullAs nullAs =
+ RexImpTable.NullAs.of(isNullable(expr));
+ return translate(expr, nullAs, storageType);
+ }
+
+ /** */
+ Expression translate(RexNode expr, RexImpTable.NullAs nullAs,
+ Type storageType) {
+ currentStorageType = storageType;
+ final Result result = expr.accept(this);
+ final Expression translated =
+ ConverterUtils.toInternal(result.valueVariable, storageType);
+ assert translated != null;
+ // When we asked for not null input that would be stored as box, avoid unboxing
+ if (RexImpTable.NullAs.NOT_POSSIBLE == nullAs
+ && translated.type.equals(storageType))
+ return translated;
+
+ return nullAs.handle(translated);
+ }
+
+ /** */
+ Expression translateCast(
+ RelDataType sourceType,
+ RelDataType targetType,
+ Expression operand) {
+ Expression convert = null;
+ switch (targetType.getSqlTypeName()) {
+ case ANY:
+ convert = operand;
+ break;
+ case DATE:
+ switch (sourceType.getSqlTypeName()) {
+ case CHAR:
+ case VARCHAR:
+ convert =
+ Expressions.call(BuiltInMethod.STRING_TO_DATE.method, operand);
+ break;
+ case TIMESTAMP:
+ convert = Expressions.convert_(
+ Expressions.call(BuiltInMethod.FLOOR_DIV.method,
+ operand, Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+ int.class);
+ break;
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_DATE.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+ }
+ break;
+ case TIME:
+ switch (sourceType.getSqlTypeName()) {
+ case CHAR:
+ case VARCHAR:
+ convert =
+ Expressions.call(BuiltInMethod.STRING_TO_TIME.method, operand);
+ break;
+ case TIME_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIME_WITH_LOCAL_TIME_ZONE_TO_TIME.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+ break;
+ case TIMESTAMP:
+ convert = Expressions.convert_(
+ Expressions.call(
+ BuiltInMethod.FLOOR_MOD.method,
+ operand,
+ Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+ int.class);
+ break;
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIME.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+ }
+ break;
+ case TIME_WITH_LOCAL_TIME_ZONE:
+ switch (sourceType.getSqlTypeName()) {
+ case CHAR:
+ case VARCHAR:
+ convert =
+ Expressions.call(BuiltInMethod.STRING_TO_TIME_WITH_LOCAL_TIME_ZONE.method, operand);
+ break;
+ case TIME:
+ convert = Expressions.call(
+ BuiltInMethod.TIME_STRING_TO_TIME_WITH_LOCAL_TIME_ZONE.method,
+ RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.UNIX_TIME_TO_STRING.method,
+ operand)),
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+ break;
+ case TIMESTAMP:
+ convert = Expressions.call(
+ BuiltInMethod.TIMESTAMP_STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+ RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+ operand)),
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+ break;
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIME_WITH_LOCAL_TIME_ZONE.method,
+ operand));
+ }
+ break;
+ case TIMESTAMP:
+ switch (sourceType.getSqlTypeName()) {
+ case CHAR:
+ case VARCHAR:
+ convert =
+ Expressions.call(BuiltInMethod.STRING_TO_TIMESTAMP.method, operand);
+ break;
+ case DATE:
+ convert = Expressions.multiply(
+ Expressions.convert_(operand, long.class),
+ Expressions.constant(DateTimeUtils.MILLIS_PER_DAY));
+ break;
+ case TIME:
+ convert =
+ Expressions.add(
+ Expressions.multiply(
+ Expressions.convert_(
+ Expressions.call(BuiltInMethod.CURRENT_DATE.method, root),
+ long.class),
+ Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+ Expressions.convert_(operand, long.class));
+ break;
+ case TIME_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIME_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+ Expressions.call(
+ BuiltInMethod.UNIX_DATE_TO_STRING.method,
+ Expressions.call(BuiltInMethod.CURRENT_DATE.method, root)),
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+ break;
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+ }
+ break;
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ switch (sourceType.getSqlTypeName()) {
+ case CHAR:
+ case VARCHAR:
+ convert =
+ Expressions.call(
+ BuiltInMethod.STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+ operand);
+ break;
+ case DATE:
+ convert = Expressions.call(
+ BuiltInMethod.TIMESTAMP_STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+ RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+ Expressions.multiply(
+ Expressions.convert_(operand, long.class),
+ Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)))),
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+ break;
+ case TIME:
+ convert = Expressions.call(
+ BuiltInMethod.TIMESTAMP_STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+ RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+ Expressions.add(
+ Expressions.multiply(
+ Expressions.convert_(
+ Expressions.call(BuiltInMethod.CURRENT_DATE.method, root),
+ long.class),
+ Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+ Expressions.convert_(operand, long.class)))),
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+ break;
+ case TIME_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIME_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+ Expressions.call(
+ BuiltInMethod.UNIX_DATE_TO_STRING.method,
+ Expressions.call(BuiltInMethod.CURRENT_DATE.method, root)),
+ operand));
+ break;
+ case TIMESTAMP:
+ convert = Expressions.call(
+ BuiltInMethod.TIMESTAMP_STRING_TO_TIMESTAMP_WITH_LOCAL_TIME_ZONE.method,
+ RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+ operand)),
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root));
+ }
+ break;
+ case BOOLEAN:
+ switch (sourceType.getSqlTypeName()) {
+ case CHAR:
+ case VARCHAR:
+ convert = Expressions.call(
+ BuiltInMethod.STRING_TO_BOOLEAN.method,
+ operand);
+ }
+ break;
+ case CHAR:
+ case VARCHAR:
+ final SqlIntervalQualifier interval =
+ sourceType.getIntervalQualifier();
+ switch (sourceType.getSqlTypeName()) {
+ case DATE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.UNIX_DATE_TO_STRING.method,
+ operand));
+ break;
+ case TIME:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.UNIX_TIME_TO_STRING.method,
+ operand));
+ break;
+ case TIME_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIME_WITH_LOCAL_TIME_ZONE_TO_STRING.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+ break;
+ case TIMESTAMP:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.UNIX_TIMESTAMP_TO_STRING.method,
+ operand));
+ break;
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_STRING.method,
+ operand,
+ Expressions.call(BuiltInMethod.TIME_ZONE.method, root)));
+ break;
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.INTERVAL_YEAR_MONTH_TO_STRING.method,
+ operand,
+ Expressions.constant(interval.timeUnitRange)));
+ break;
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.INTERVAL_DAY_TIME_TO_STRING.method,
+ operand,
+ Expressions.constant(interval.timeUnitRange),
+ Expressions.constant(
+ interval.getFractionalSecondPrecision(
+ typeFactory.getTypeSystem()))));
+ break;
+ case BOOLEAN:
+ convert = RexImpTable.optimize2(
+ operand,
+ Expressions.call(
+ BuiltInMethod.BOOLEAN_TO_STRING.method,
+ operand));
+ break;
+ }
+ }
+ if (convert == null)
+ convert = ConverterUtils.convert(operand, typeFactory.getJavaClass(targetType));
+
+ // Going from anything to CHAR(n) or VARCHAR(n), make sure value is no
+ // longer than n.
+ boolean pad = false;
+ boolean truncate = true;
+ switch (targetType.getSqlTypeName()) {
+ case CHAR:
+ case BINARY:
+ pad = true;
+ // fall through
+ case VARCHAR:
+ case VARBINARY:
+ final int targetPrecision = targetType.getPrecision();
+ if (targetPrecision >= 0) {
+ switch (sourceType.getSqlTypeName()) {
+ case CHAR:
+ case VARCHAR:
+ case BINARY:
+ case VARBINARY:
+ // If this is a widening cast, no need to truncate.
+ final int sourcePrecision = sourceType.getPrecision();
+ if (SqlTypeUtil.comparePrecision(sourcePrecision, targetPrecision) <= 0)
+ truncate = false;
+
+ // If this is a widening cast, no need to pad.
+ if (SqlTypeUtil.comparePrecision(sourcePrecision, targetPrecision) >= 0)
+ pad = false;
+
+ // fall through
+ default:
+ if (truncate || pad) {
+ convert =
+ Expressions.call(
+ pad
+ ? BuiltInMethod.TRUNCATE_OR_PAD.method
+ : BuiltInMethod.TRUNCATE.method,
+ convert,
+ Expressions.constant(targetPrecision));
+ }
+ }
+ }
+ break;
+ case TIMESTAMP:
+ int targetScale = targetType.getScale();
+ if (targetScale == RelDataType.SCALE_NOT_SPECIFIED)
+ targetScale = 0;
+
+ if (targetScale < sourceType.getScale()) {
+ convert =
+ Expressions.call(
+ BuiltInMethod.ROUND_LONG.method,
+ convert,
+ Expressions.constant(
+ (long)Math.pow(10, 3 - targetScale)));
+ }
+ break;
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ switch (sourceType.getSqlTypeName().getFamily()) {
+ case NUMERIC:
+ final BigDecimal multiplier = targetType.getSqlTypeName().getEndUnit().multiplier;
+ final BigDecimal divider = BigDecimal.ONE;
+ convert = RexImpTable.multiplyDivide(convert, multiplier, divider);
+ }
+ }
+ return scaleIntervalToNumber(sourceType, targetType, convert);
+ }
+
+ /**
+ * Dereferences an expression if it is a {@link RexLocalRef}.
+ */
+ public RexNode deref(RexNode expr) {
+ if (expr instanceof RexLocalRef) {
+ RexLocalRef ref = (RexLocalRef)expr;
+ final RexNode e2 = program.getExprList().get(ref.getIndex());
+ assert ref.getType().equals(e2.getType());
+ return e2;
+ }
+
+ return expr;
+ }
+
+ /**
+ * Translates a literal.
+ *
+ * @throws ControlFlowException if literal is null but {@code nullAs} is {@link RexImpTable.NullAs#NOT_POSSIBLE}.
+ */
+ public static Expression translateLiteral(
+ RexLiteral literal,
+ RelDataType type,
+ JavaTypeFactory typeFactory,
+ RexImpTable.NullAs nullAs) {
+ if (literal.isNull()) {
+ switch (nullAs) {
+ case TRUE:
+ case IS_NULL:
+ return RexImpTable.TRUE_EXPR;
+ case FALSE:
+ case IS_NOT_NULL:
+ return RexImpTable.FALSE_EXPR;
+ case NOT_POSSIBLE:
+ throw new ControlFlowException();
+ case NULL:
+ default:
+ return RexImpTable.NULL_EXPR;
+ }
+ }
+ else {
+ switch (nullAs) {
+ case IS_NOT_NULL:
+ return RexImpTable.TRUE_EXPR;
+ case IS_NULL:
+ return RexImpTable.FALSE_EXPR;
+ }
+ }
+ Type javaClass = typeFactory.getJavaClass(type);
+ final Object value2;
+ switch (literal.getType().getSqlTypeName()) {
+ case DECIMAL:
+ final BigDecimal bd = literal.getValueAs(BigDecimal.class);
+ if (javaClass == float.class)
+ return Expressions.constant(bd, javaClass);
+ else if (javaClass == double.class)
+ return Expressions.constant(bd, javaClass);
+ assert javaClass == BigDecimal.class;
+ return Expressions.new_(BigDecimal.class,
+ Expressions.constant(bd.toString()));
+ case DATE:
+ case TIME:
+ case TIME_WITH_LOCAL_TIME_ZONE:
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ value2 = literal.getValueAs(Integer.class);
+ javaClass = int.class;
+ break;
+ case TIMESTAMP:
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ value2 = literal.getValueAs(Long.class);
+ javaClass = long.class;
+ break;
+ case CHAR:
+ case VARCHAR:
+ value2 = literal.getValueAs(String.class);
+ break;
+ case BINARY:
+ case VARBINARY:
+ return Expressions.new_(
+ ByteString.class,
+ Expressions.constant(
+ literal.getValueAs(byte[].class),
+ byte[].class));
+ case GEOMETRY:
+ final Geometries.Geom geom = literal.getValueAs(Geometries.Geom.class);
+ final String wkt = GeoFunctions.ST_AsWKT(geom);
+ return Expressions.call(null, BuiltInMethod.ST_GEOM_FROM_TEXT.method,
+ Expressions.constant(wkt));
+ case SYMBOL:
+ value2 = literal.getValueAs(Enum.class);
+ javaClass = value2.getClass();
+ break;
+ default:
+ final Primitive primitive = Primitive.ofBoxOr(javaClass);
+ final Comparable value = literal.getValueAs(Comparable.class);
+
+ value2 = primitive != null && value instanceof Number ? primitive.number((Number)value) : value;
+ }
+ return Expressions.constant(value2, javaClass);
+ }
+
+ /** */
+ public List<Expression> translateList(
+ List<RexNode> operandList,
+ RexImpTable.NullAs nullAs) {
+ return translateList(operandList, nullAs,
+ ConverterUtils.internalTypes(operandList));
+ }
+
+ /** */
+ public List<Expression> translateList(
+ List<RexNode> operandList,
+ RexImpTable.NullAs nullAs,
+ List<? extends Type> storageTypes) {
+ final List<Expression> list = new ArrayList<>();
+ for (Pair<RexNode, ? extends Type> e : Pair.zip(operandList, storageTypes))
+ list.add(translate(e.left, nullAs, e.right));
+
+ return list;
+ }
+
+ /**
+ * Translates the list of {@code RexNode}, using the default output types. This might be suboptimal in terms of
+ * additional box-unbox when you use the translation later. If you know the java class that will be used to store
+ * the results, use {@link RexToLixTranslator#translateList(List, List)} version.
+ *
+ * @param operandList list of RexNodes to translate
+ * @return translated expressions
+ */
+ public List<Expression> translateList(List<? extends RexNode> operandList) {
+ return translateList(operandList, ConverterUtils.internalTypes(operandList));
+ }
+
+ /**
+ * Translates the list of {@code RexNode}, while optimizing for output storage. For instance, if the result of
+ * translation is going to be stored in {@code Object[]}, and the input is {@code Object[]} as well, then translator
+ * will avoid casting, boxing, etc.
+ *
+ * @param operandList list of RexNodes to translate
+ * @param storageTypes hints of the java classes that will be used to store translation results. Use null to use
+ * default storage type
+ * @return translated expressions
+ */
+ public List<Expression> translateList(List<? extends RexNode> operandList,
+ List<? extends Type> storageTypes) {
+ final List<Expression> list = new ArrayList<>(operandList.size());
+
+ for (int i = 0; i < operandList.size(); i++) {
+ RexNode rex = operandList.get(i);
+ Type desiredType = null;
+ if (storageTypes != null)
+ desiredType = storageTypes.get(i);
+
+ final Expression translate = translate(rex, desiredType);
+ list.add(translate);
+ // desiredType is still a hint, thus we might get any kind of output
+ // (boxed or not) when hint was provided.
+ // It is favourable to get the type matching desired type
+ if (desiredType == null && !isNullable(rex)) {
+ assert !Primitive.isBox(translate.getType())
+ : "Not-null boxed primitive should come back as primitive: "
+ + rex + ", " + translate.getType();
+ }
+ }
+ return list;
+ }
+
+ /**
+ * Returns whether an expression is nullable.
+ *
+ * @param e Expression
+ * @return Whether expression is nullable
+ */
+ public boolean isNullable(RexNode e) {
+ return e.getType().isNullable();
+ }
+
+ /** */
+ public RexToLixTranslator setBlock(BlockBuilder block) {
+ if (block == list)
+ return this;
+
+ return new RexToLixTranslator(program, typeFactory, root, inputGetter,
+ block, builder, conformance, correlates);
+ }
+
+ /** */
+ public RexToLixTranslator setCorrelates(
+ Function1<String, InputGetter> correlates) {
+ if (this.correlates == correlates)
+ return this;
+
+ return new RexToLixTranslator(program, typeFactory, root, inputGetter, list,
+ builder, conformance, correlates);
+ }
+
+ /** */
+ public Expression getRoot() {
+ return root;
+ }
+
+ /** */
+ private static Expression scaleIntervalToNumber(
+ RelDataType sourceType,
+ RelDataType targetType,
+ Expression operand) {
+ switch (targetType.getSqlTypeName().getFamily()) {
+ case NUMERIC:
+ switch (sourceType.getSqlTypeName()) {
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ // Scale to the given field.
+ final BigDecimal multiplier = BigDecimal.ONE;
+ final BigDecimal divider =
+ sourceType.getSqlTypeName().getEndUnit().multiplier;
+ return RexImpTable.multiplyDivide(operand, multiplier, divider);
+ }
+ }
+ return operand;
+ }
+
+ /**
+ * Visit {@code RexInputRef}. If it has never been visited under current storage type before, {@code
+ * RexToLixTranslator} generally produces three lines of code. For example, when visiting a column (named
+ * commission) in table Employee, the generated code snippet is: {@code final Employee current =(Employee)
+ * inputEnumerator.current(); final Integer input_value = current.commission; final boolean input_isNull =
+ * input_value == null; }
+ */
+ @Override public Result visitInputRef(RexInputRef inputRef) {
+ final Pair<RexNode, Type> key = Pair.of(inputRef, currentStorageType);
+ // If the RexInputRef has been visited under current storage type already,
+ // it is not necessary to visit it again, just return the result.
+ if (rexWithStorageTypeResultMap.containsKey(key))
+ return rexWithStorageTypeResultMap.get(key);
+
+ // Generate one line of code to get the input, e.g.,
+ // "final Employee current =(Employee) inputEnumerator.current();"
+ final Expression valueExpression = inputGetter.field(
+ list, inputRef.getIndex(), currentStorageType);
+
+ // Generate one line of code for the value of RexInputRef, e.g.,
+ // "final Integer input_value = current.commission;"
+ final ParameterExpression valueVariable =
+ Expressions.parameter(
+ valueExpression.getType(), list.newName("input_value"));
+ list.add(Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+
+ // Generate one line of code to check whether RexInputRef is null, e.g.,
+ // "final boolean input_isNull = input_value == null;"
+ final Expression isNullExpression = checkNull(valueVariable);
+ final ParameterExpression isNullVariable =
+ Expressions.parameter(
+ Boolean.TYPE, list.newName("input_isNull"));
+ list.add(Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+
+ final Result result = new Result(isNullVariable, valueVariable);
+
+ // Cache <RexInputRef, currentStorageType>'s result
+ rexWithStorageTypeResultMap.put(key, result);
+
+ return new Result(isNullVariable, valueVariable);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitLocalRef(RexLocalRef localRef) {
+ return deref(localRef).accept(this);
+ }
+
+ /**
+ * Visit {@code RexLiteral}. If it has never been visited before, {@code RexToLixTranslator} will generate two lines
+ * of code. For example, when visiting a primitive int (10), the generated code snippet is: {@code final int
+ * literal_value = 10; final boolean literal_isNull = false; }
+ */
+ @Override public Result visitLiteral(RexLiteral literal) {
+ // If the RexLiteral has been visited already, just return the result
+ if (rexResultMap.containsKey(literal))
+ return rexResultMap.get(literal);
+
+ // Generate one line of code for the value of RexLiteral, e.g.,
+ // "final int literal_value = 10;"
+ final Expression valueExpression = literal.isNull()
+ // Note: even for null literal, we can't loss its type information
+ ? getTypedNullLiteral(literal)
+ : translateLiteral(literal, literal.getType(),
+ typeFactory, RexImpTable.NullAs.NOT_POSSIBLE);
+ final ParameterExpression valueVariable =
+ Expressions.parameter(valueExpression.getType(),
+ list.newName("literal_value"));
+ list.add(Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+
+ // Generate one line of code to check whether RexLiteral is null, e.g.,
+ // "final boolean literal_isNull = false;"
+ final Expression isNullExpression =
+ literal.isNull() ? RexImpTable.TRUE_EXPR : RexImpTable.FALSE_EXPR;
+ final ParameterExpression isNullVariable = Expressions.parameter(
+ Boolean.TYPE, list.newName("literal_isNull"));
+ list.add(Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+
+ // Maintain the map from valueVariable (ParameterExpression) to real Expression
+ literalMap.put(valueVariable, valueExpression);
+ final Result result = new Result(isNullVariable, valueVariable);
+ // Cache RexLiteral's result
+ rexResultMap.put(literal, result);
+ return result;
+ }
+
+ /**
+ * Returns an {@code Expression} for null literal without losing its type information.
+ */
+ private ConstantExpression getTypedNullLiteral(RexLiteral literal) {
+ assert literal.isNull();
+ Type javaClass = typeFactory.getJavaClass(literal.getType());
+ switch (literal.getType().getSqlTypeName()) {
+ case DATE:
+ case TIME:
+ case TIME_WITH_LOCAL_TIME_ZONE:
+ case INTERVAL_YEAR:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_MONTH:
+ javaClass = Integer.class;
+ break;
+ case TIMESTAMP:
+ case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+ case INTERVAL_DAY:
+ case INTERVAL_DAY_HOUR:
+ case INTERVAL_DAY_MINUTE:
+ case INTERVAL_DAY_SECOND:
+ case INTERVAL_HOUR:
+ case INTERVAL_HOUR_MINUTE:
+ case INTERVAL_HOUR_SECOND:
+ case INTERVAL_MINUTE:
+ case INTERVAL_MINUTE_SECOND:
+ case INTERVAL_SECOND:
+ javaClass = Long.class;
+ break;
+ }
+ return javaClass == null || javaClass == Void.class
+ ? RexImpTable.NULL_EXPR
+ : Expressions.constant(null, javaClass);
+ }
+
+ /**
+ * Visit {@code RexCall}. For most {@code SqlOperator}s, we can get the implementor from {@code RexImpTable}.
+ * Several operators (e.g., CaseWhen) with special semantics need to be implemented separately.
+ */
+ @Override public Result visitCall(RexCall call) {
+ if (rexResultMap.containsKey(call))
+ return rexResultMap.get(call);
+
+ final SqlOperator operator = call.getOperator();
+ if (operator == CASE)
+ return implementCaseWhen(call);
+
+ if (operator == SEARCH)
+ return RexUtil.expandSearch(builder, program, call).accept(this);
+
+ final RexImpTable.RexCallImplementor implementor =
+ RexImpTable.INSTANCE.get(operator);
+ if (implementor == null)
+ throw new RuntimeException("cannot translate call " + call);
+
+ final List<RexNode> operandList = call.getOperands();
+ final List<Type> storageTypes = ConverterUtils.internalTypes(operandList);
+ final List<Result> operandResults = new ArrayList<>();
+ for (int i = 0; i < operandList.size(); i++) {
+ final Result operandResult =
+ implementCallOperand(operandList.get(i), storageTypes.get(i), this);
+ operandResults.add(operandResult);
+ }
+ callOperandResultMap.put(call, operandResults);
+ final Result result = implementor.implement(this, call, operandResults);
+ rexResultMap.put(call, result);
+ return result;
+ }
+
+ /** */
+ private static Result implementCallOperand(final RexNode operand,
+ final Type storageType, final RexToLixTranslator translator) {
+ final Type originalStorageType = translator.currentStorageType;
+ translator.currentStorageType = storageType;
+ Result operandResult = operand.accept(translator);
+ if (storageType != null)
+ operandResult = translator.toInnerStorageType(operandResult, storageType);
+ translator.currentStorageType = originalStorageType;
+ return operandResult;
+ }
+
+ /** */
+ private static Expression implementCallOperand2(final RexNode operand,
+ final Type storageType, final RexToLixTranslator translator) {
+ final Type originalStorageType = translator.currentStorageType;
+ translator.currentStorageType = storageType;
+ final Expression result = translator.translate(operand);
+ translator.currentStorageType = originalStorageType;
+ return result;
+ }
+
+ /**
+ * The CASE operator is SQL’s way of handling if/then logic. Different with other {@code RexCall}s, it is not safe
+ * to implement its operands first. For example: {@code select case when s=0 then false else 100/s > 0 end from
+ * (values (1),(0)) ax(s); }
+ */
+ private Result implementCaseWhen(RexCall call) {
+ final Type returnType = typeFactory.getJavaClass(call.getType());
+ final ParameterExpression valueVariable =
+ Expressions.parameter(returnType,
+ list.newName("case_when_value"));
+ list.add(Expressions.declare(0, valueVariable, null));
+ final List<RexNode> operandList = call.getOperands();
+ implementRecursively(this, operandList, valueVariable, 0);
+ final Expression isNullExpression = checkNull(valueVariable);
+ final ParameterExpression isNullVariable =
+ Expressions.parameter(
+ Boolean.TYPE, list.newName("case_when_isNull"));
+ list.add(Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+ final Result result = new Result(isNullVariable, valueVariable);
+ rexResultMap.put(call, result);
+ return result;
+ }
+
+ /**
+ * Case statements of the form: {@code CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END}. When {@code a = true},
+ * returns {@code b}; when {@code c = true}, returns {@code d}; else returns {@code e}.
+ *
+ * <p>We generate code that looks like:
+ *
+ * <blockquote><pre>
+ * int case_when_value;
+ * ......code for a......
+ * if (!a_isNull && a_value) {
+ * ......code for b......
+ * case_when_value = res(b_isNull, b_value);
+ * } else {
+ * ......code for c......
+ * if (!c_isNull && c_value) {
+ * ......code for d......
+ * case_when_value = res(d_isNull, d_value);
+ * } else {
+ * ......code for e......
+ * case_when_value = res(e_isNull, e_value);
+ * }
+ * }
+ * </pre></blockquote>
+ */
+ private void implementRecursively(final RexToLixTranslator currentTranslator,
+ final List<RexNode> operandList, final ParameterExpression valueVariable, int pos) {
+ final BlockBuilder currentBlockBuilder = currentTranslator.getBlockBuilder();
+ final List<Type> storageTypes = ConverterUtils.internalTypes(operandList);
+ // [ELSE] clause
+ if (pos == operandList.size() - 1) {
+ Expression res = implementCallOperand2(operandList.get(pos),
+ storageTypes.get(pos), currentTranslator);
+ currentBlockBuilder.add(
+ Expressions.statement(
+ Expressions.assign(valueVariable,
+ ConverterUtils.convert(res, valueVariable.getType()))));
+ return;
+ }
+ // Condition code: !a_isNull && a_value
+ final RexNode testerNode = operandList.get(pos);
+ final Result testerResult = implementCallOperand(testerNode,
+ storageTypes.get(pos), currentTranslator);
+ final Expression tester = Expressions.andAlso(
+ Expressions.not(testerResult.isNullVariable),
+ testerResult.valueVariable);
+ // Code for {if} branch
+ final RexNode ifTrueNode = operandList.get(pos + 1);
+ final BlockBuilder ifTrueBlockBuilder =
+ new BlockBuilder(true, currentBlockBuilder);
+ final RexToLixTranslator ifTrueTranslator =
+ currentTranslator.setBlock(ifTrueBlockBuilder);
+ final Expression ifTrueRes = implementCallOperand2(ifTrueNode,
+ storageTypes.get(pos + 1), ifTrueTranslator);
+ // Assign the value: case_when_value = ifTrueRes
+ ifTrueBlockBuilder.add(
+ Expressions.statement(
+ Expressions.assign(valueVariable,
+ ConverterUtils.convert(ifTrueRes, valueVariable.getType()))));
+ final BlockStatement ifTrue = ifTrueBlockBuilder.toBlock();
+ // There is no [ELSE] clause
+ if (pos + 1 == operandList.size() - 1) {
+ currentBlockBuilder.add(
+ Expressions.ifThen(tester, ifTrue));
+ return;
+ }
+ // Generate code for {else} branch recursively
+ final BlockBuilder ifFalseBlockBuilder =
+ new BlockBuilder(true, currentBlockBuilder);
+ final RexToLixTranslator ifFalseTranslator =
+ currentTranslator.setBlock(ifFalseBlockBuilder);
+ implementRecursively(ifFalseTranslator, operandList, valueVariable, pos + 2);
+ final BlockStatement ifFalse = ifFalseBlockBuilder.toBlock();
+ currentBlockBuilder.add(
+ Expressions.ifThenElse(tester, ifTrue, ifFalse));
+ }
+
+ /** */
+ private Result toInnerStorageType(final Result result, final Type storageType) {
+ final Expression valueExpression =
+ ConverterUtils.toInternal(result.valueVariable, storageType);
+ if (valueExpression.equals(result.valueVariable))
+ return result;
+
+ final ParameterExpression valueVariable =
+ Expressions.parameter(
+ valueExpression.getType(),
+ list.newName(result.valueVariable.name + "_inner_type"));
+ list.add(Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+ final ParameterExpression isNullVariable = result.isNullVariable;
+ return new Result(isNullVariable, valueVariable);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitDynamicParam(RexDynamicParam dynamicParam) {
+ final Pair<RexNode, Type> key = Pair.of(dynamicParam, currentStorageType);
+ if (rexWithStorageTypeResultMap.containsKey(key))
+ return rexWithStorageTypeResultMap.get(key);
+
+ final Type storageType = currentStorageType != null
+ ? currentStorageType : typeFactory.getJavaClass(dynamicParam.getType());
+ final Expression valueExpression = ConverterUtils.convert(
+ Expressions.call(root, BuiltInMethod.DATA_CONTEXT_GET.method,
+ Expressions.constant("?" + dynamicParam.getIndex())),
+ storageType);
+ final ParameterExpression valueVariable =
+ Expressions.parameter(valueExpression.getType(), list.newName("value_dynamic_param"));
+ list.add(Expressions.declare(Modifier.FINAL, valueVariable, valueExpression));
+ final ParameterExpression isNullVariable =
+ Expressions.parameter(Boolean.TYPE, list.newName("isNull_dynamic_param"));
+ list.add(Expressions.declare(Modifier.FINAL, isNullVariable, checkNull(valueVariable)));
+ final Result result = new Result(isNullVariable, valueVariable);
+ rexWithStorageTypeResultMap.put(key, result);
+ return result;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitFieldAccess(RexFieldAccess fieldAccess) {
+ final Pair<RexNode, Type> key = Pair.of(fieldAccess, currentStorageType);
+ if (rexWithStorageTypeResultMap.containsKey(key))
+ return rexWithStorageTypeResultMap.get(key);
+
+ final RexNode target = deref(fieldAccess.getReferenceExpr());
+ int fieldIndex = fieldAccess.getField().getIndex();
+ String fieldName = fieldAccess.getField().getName();
+ switch (target.getKind()) {
+ case CORREL_VARIABLE:
+ if (correlates == null) {
+ throw new RuntimeException("Cannot translate " + fieldAccess
+ + " since correlate variables resolver is not defined");
+ }
+ final RexToLixTranslator.InputGetter getter =
+ correlates.apply(((RexVariable)target).getName());
+ final Expression input = getter.field(
+ list, fieldIndex, currentStorageType);
+ final Expression condition = checkNull(input);
+ final ParameterExpression valueVariable =
+ Expressions.parameter(input.getType(), list.newName("corInp_value"));
+ list.add(Expressions.declare(Modifier.FINAL, valueVariable, input));
+ final ParameterExpression isNullVariable =
+ Expressions.parameter(Boolean.TYPE, list.newName("corInp_isNull"));
+ final Expression isNullExpression = Expressions.condition(
+ condition,
+ RexImpTable.TRUE_EXPR,
+ checkNull(valueVariable));
+ list.add(Expressions.declare(Modifier.FINAL, isNullVariable, isNullExpression));
+ final Result result1 = new Result(isNullVariable, valueVariable);
+ rexWithStorageTypeResultMap.put(key, result1);
+ return result1;
+ default:
+ RexNode rxIndex =
+ builder.makeLiteral(fieldIndex, typeFactory.createType(int.class), true);
+ RexNode rxName =
+ builder.makeLiteral(fieldName, typeFactory.createType(String.class), true);
+ RexCall accessCall = (RexCall)builder.makeCall(
+ fieldAccess.getType(), SqlStdOperatorTable.STRUCT_ACCESS,
+ ImmutableList.of(target, rxIndex, rxName));
+ final Result result2 = accessCall.accept(this);
+ rexWithStorageTypeResultMap.put(key, result2);
+ return result2;
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitOver(RexOver over) {
+ throw new RuntimeException("cannot translate expression " + over);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitCorrelVariable(RexCorrelVariable correlVariable) {
+ throw new RuntimeException("Cannot translate " + correlVariable
+ + ". Correlated variables should always be referenced by field access");
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitRangeRef(RexRangeRef rangeRef) {
+ throw new RuntimeException("cannot translate expression " + rangeRef);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitSubQuery(RexSubQuery subQuery) {
+ throw new RuntimeException("cannot translate expression " + subQuery);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitTableInputRef(RexTableInputRef fieldRef) {
+ throw new RuntimeException("cannot translate expression " + fieldRef);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Result visitPatternFieldRef(RexPatternFieldRef fieldRef) {
+ return visitInputRef(fieldRef);
+ }
+
+ /** */
+ Expression checkNull(Expression expr) {
+ if (Primitive.flavor(expr.getType()) == Primitive.Flavor.PRIMITIVE)
+ return RexImpTable.FALSE_EXPR;
+
+ return Expressions.equal(expr, RexImpTable.NULL_EXPR);
+ }
+
+ /** */
+ Expression checkNotNull(Expression expr) {
+ if (Primitive.flavor(expr.getType()) == Primitive.Flavor.PRIMITIVE)
+ return RexImpTable.TRUE_EXPR;
+
+ return Expressions.notEqual(expr, RexImpTable.NULL_EXPR);
+ }
+
+ /** */
+ BlockBuilder getBlockBuilder() {
+ return list;
+ }
+
+ /** */
+ Expression getLiteral(Expression literalVariable) {
+ return literalMap.get(literalVariable);
+ }
+
+ /** Returns the value of a literal. */
+ Object getLiteralValue(Expression expr) {
+ if (expr instanceof ParameterExpression) {
+ final Expression constantExpr = literalMap.get(expr);
+ return getLiteralValue(constantExpr);
+ }
+ if (expr instanceof ConstantExpression)
+ return ((ConstantExpression)expr).value;
+
+ return null;
+ }
+
+ /** */
+ List<Result> getCallOperandResult(RexCall call) {
+ return callOperandResultMap.get(call);
+ }
+
+ /** Translates a field of an input to an expression. */
+ public interface InputGetter {
+ /** */
+ Expression field(BlockBuilder list, int index, Type storageType);
+ }
+
+ /** Result of translating a {@code RexNode}. */
+ public static class Result {
+ /** */
+ final ParameterExpression isNullVariable;
+
+ /** */
+ final ParameterExpression valueVariable;
+
+ /** */
+ public Result(ParameterExpression isNullVariable,
+ ParameterExpression valueVariable) {
+ this.isNullVariable = isNullVariable;
+ this.valueVariable = valueVariable;
+ }
+ }
+
+ /**
+ * Handle checked Exceptions declared in Method. In such case,
+ * method call should be wrapped in a try...catch block.
+ * "
+ * final Type method_call;
+ * try {
+ * method_call = callExpr
+ * } catch (Exception e) {
+ * throw new RuntimeException(e);
+ * }
+ * "
+ */
+ Expression handleMethodCheckedExceptions(Expression callExpr) {
+ // Try statement
+ ParameterExpression methodCall = Expressions.parameter(
+ callExpr.getType(), list.newName("method_call"));
+ list.add(Expressions.declare(Modifier.FINAL, methodCall, null));
+ Statement st = Expressions.statement(Expressions.assign(methodCall, callExpr));
+ // Catch Block, wrap checked exception in unchecked exception
+ ParameterExpression e = Expressions.parameter(0, Exception.class, "e");
+ Expression uncheckedException = Expressions.new_(RuntimeException.class, e);
+ CatchBlock cb = Expressions.catch_(e, Expressions.throw_(uncheckedException));
+ list.add(Expressions.tryCatch(st, cb));
+ return methodCall;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
similarity index 60%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
index ce7d926..9fc3d9e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/Scalar.java
@@ -14,27 +14,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite;
-/** Stubs */
-public class Stubs {
- /** */
- public static int intFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
-
- /** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+package org.apache.ignite.internal.processors.query.calcite.exec.exp;
- /** */
- public static boolean boolFoo(Object... args) {
- return args == null;
- }
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+/** */
+public interface Scalar {
/** */
- public static String stringFoo(Object... args) {
- return args == null ? "null" : "not null";
- }
+ void execute(ExecutionContext ctx, Object in, Object out);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
similarity index 61%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
index ce7d926..b4b9b91 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulator.java
@@ -14,27 +14,29 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite;
-/** Stubs */
-public class Stubs {
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+/** */
+public interface Accumulator extends Serializable {
+ /** */
+ void add(Object... args);
+
/** */
- public static int intFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+ void apply(Accumulator other);
/** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+ Object end();
/** */
- public static boolean boolFoo(Object... args) {
- return args == null;
- }
+ List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory);
/** */
- public static String stringFoo(Object... args) {
- return args == null ? "null" : "not null";
- }
+ RelDataType returnType(IgniteTypeFactory typeFactory);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
similarity index 63%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
index ce7d926..7d7702b 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorWrapper.java
@@ -14,27 +14,20 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite;
-/** Stubs */
-public class Stubs {
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+/** */
+public interface AccumulatorWrapper<Row> {
/** */
- public static int intFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+ void add(Row row);
/** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+ Object end();
/** */
- public static boolean boolFoo(Object... args) {
- return args == null;
- }
+ void apply(Accumulator accumulator);
/** */
- public static String stringFoo(Object... args) {
- return args == null ? "null" : "not null";
- }
+ Accumulator accumulator();
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
new file mode 100644
index 0000000..4c61423
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
@@ -0,0 +1,1068 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Supplier;
+
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+
+import static org.apache.calcite.sql.type.SqlTypeName.ANY;
+import static org.apache.calcite.sql.type.SqlTypeName.BIGINT;
+import static org.apache.calcite.sql.type.SqlTypeName.DECIMAL;
+import static org.apache.calcite.sql.type.SqlTypeName.DOUBLE;
+import static org.apache.calcite.sql.type.SqlTypeName.INTEGER;
+import static org.apache.calcite.sql.type.SqlTypeName.VARCHAR;
+import static org.apache.ignite.internal.util.ArrayUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class Accumulators {
+ /** */
+ public static Supplier<Accumulator> accumulatorFactory(AggregateCall call) {
+ if (!call.isDistinct())
+ return accumulatorFunctionFactory(call);
+
+ Supplier<Accumulator> fac = accumulatorFunctionFactory(call);
+
+ return () -> new DistinctAccumulator(fac);
+ }
+
+ /** */
+ public static Supplier<Accumulator> accumulatorFunctionFactory(AggregateCall call) {
+ switch (call.getAggregation().getName()) {
+ case "COUNT":
+ return LongCount.FACTORY;
+ case "AVG":
+ return avgFactory(call);
+ case "SUM":
+ return sumFactory(call);
+ case "$SUM0":
+ return sumEmptyIsZeroFactory(call);
+ case "MIN":
+ return minFactory(call);
+ case "MAX":
+ return maxFactory(call);
+ case "SINGLE_VALUE":
+ return SingleVal.FACTORY;
+ default:
+ throw new AssertionError(call.getAggregation().getName());
+ }
+ }
+
+ /** */
+ private static Supplier<Accumulator> avgFactory(AggregateCall call) {
+ switch (call.type.getSqlTypeName()) {
+ case BIGINT:
+ case DECIMAL:
+ return DecimalAvg.FACTORY;
+ case DOUBLE:
+ case REAL:
+ case FLOAT:
+ case INTEGER:
+ default:
+ return DoubleAvg.FACTORY;
+ }
+ }
+
+ /** */
+ private static Supplier<Accumulator> sumFactory(AggregateCall call) {
+ switch (call.type.getSqlTypeName()) {
+ case DOUBLE:
+ case REAL:
+ case FLOAT:
+ return DoubleSum.FACTORY;
+ case DECIMAL:
+ return DecimalSum.FACTORY;
+ case INTEGER:
+ return IntSum.FACTORY;
+ case BIGINT:
+ default:
+ return LongSum.FACTORY;
+ }
+ }
+
+ /** */
+ private static Supplier<Accumulator> sumEmptyIsZeroFactory(AggregateCall call) {
+ switch (call.type.getSqlTypeName()) {
+ case DOUBLE:
+ case REAL:
+ case FLOAT:
+ return DoubleSumEmptyIsZero.FACTORY;
+ case DECIMAL:
+ return DecimalSumEmptyIsZero.FACTORY;
+ case INTEGER:
+ return IntSumEmptyIsZero.FACTORY;
+ case BIGINT:
+ default:
+ return LongSumEmptyIsZero.FACTORY;
+ }
+ }
+
+ /** */
+ private static Supplier<Accumulator> minFactory(AggregateCall call) {
+ switch (call.type.getSqlTypeName()) {
+ case DOUBLE:
+ case REAL:
+ case FLOAT:
+ return DoubleMinMax.MIN_FACTORY;
+ case DECIMAL:
+ return DecimalMinMax.MIN_FACTORY;
+ case INTEGER:
+ return IntMinMax.MIN_FACTORY;
+ case CHAR:
+ case VARCHAR:
+ return VarCharMinMax.MIN_FACTORY;
+ case BIGINT:
+ default:
+ return LongMinMax.MIN_FACTORY;
+ }
+ }
+
+ /** */
+ private static Supplier<Accumulator> maxFactory(AggregateCall call) {
+ switch (call.type.getSqlTypeName()) {
+ case DOUBLE:
+ case REAL:
+ case FLOAT:
+ return DoubleMinMax.MAX_FACTORY;
+ case DECIMAL:
+ return DecimalMinMax.MAX_FACTORY;
+ case INTEGER:
+ return IntMinMax.MAX_FACTORY;
+ case CHAR:
+ case VARCHAR:
+ return VarCharMinMax.MAX_FACTORY;
+ case BIGINT:
+ default:
+ return LongMinMax.MAX_FACTORY;
+ }
+ }
+
+ /** */
+ private static class SingleVal implements Accumulator {
+ /** */
+ private Object holder;
+
+ /** */
+ private boolean touched;
+
+ /** */
+ public static final Supplier<Accumulator> FACTORY = SingleVal::new;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ assert args.length == 1 : args.length;
+
+ if (touched)
+ throw new IllegalArgumentException("Subquery returned more than 1 value.");
+
+ touched = true;
+
+ holder = args[0];
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ assert holder == null : "sudden apply for: " + other + " on SingleVal";
+
+ holder = ((SingleVal)other).holder;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return holder;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(ANY), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createSqlType(ANY);
+ }
+ }
+
+ /** */
+ public static class DecimalAvg implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = DecimalAvg::new;
+
+ /** */
+ private BigDecimal sum = BigDecimal.ZERO;
+
+ /** */
+ private BigDecimal cnt = BigDecimal.ZERO;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ BigDecimal in = (BigDecimal)args[0];
+
+ if (in == null)
+ return;
+
+ sum = sum.add(in);
+ cnt = cnt.add(BigDecimal.ONE);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DecimalAvg other0 = (DecimalAvg)other;
+
+ sum = sum.add(other0.sum);
+ cnt = cnt.add(other0.cnt);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return cnt.compareTo(BigDecimal.ZERO) == 0 ? null : sum.divide(cnt, MathContext.DECIMAL64);
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true);
+ }
+ }
+
+ /** */
+ public static class DoubleAvg implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = DoubleAvg::new;
+
+ /** */
+ private double sum;
+
+ /** */
+ private long cnt;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Double in = (Double)args[0];
+
+ if (in == null)
+ return;
+
+ sum += in;
+ cnt++;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DoubleAvg other0 = (DoubleAvg)other;
+
+ sum += other0.sum;
+ cnt += other0.cnt;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return cnt > 0 ? sum / cnt : null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true);
+ }
+ }
+
+ /** */
+ private static class LongCount implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = LongCount::new;
+
+ /** */
+ private long cnt;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ assert nullOrEmpty(args) || args.length == 1;
+
+ if (nullOrEmpty(args) || args[0] != null)
+ cnt++;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ LongCount other0 = (LongCount)other;
+ cnt += other0.cnt;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return cnt;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(ANY), false));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createSqlType(BIGINT);
+ }
+ }
+
+ /** */
+ private static class DoubleSum implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = DoubleSum::new;
+
+ /** */
+ private double sum;
+
+ /** */
+ private boolean empty = true;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Double in = (Double)args[0];
+
+ if (in == null)
+ return;
+
+ empty = false;
+ sum += in;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DoubleSum other0 = (DoubleSum)other;
+
+ if (other0.empty)
+ return;
+
+ empty = false;
+ sum += other0.sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return empty ? null : sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true);
+ }
+ }
+
+ /** */
+ private static class IntSum implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = IntSum::new;
+
+ /** */
+ private int sum;
+
+ /** */
+ private boolean empty = true;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Integer in = (Integer)args[0];
+
+ if (in == null)
+ return;
+
+ empty = false;
+ sum += in;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ IntSum other0 = (IntSum)other;
+
+ if (other0.empty)
+ return;
+
+ empty = false;
+ sum += other0.sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return empty ? null : sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true);
+ }
+ }
+
+ /** */
+ private static class LongSum implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = LongSum::new;
+
+ /** */
+ private long sum;
+
+ /** */
+ private boolean empty = true;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Long in = (Long)args[0];
+
+ if (in == null)
+ return;
+
+ empty = false;
+ sum += in;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ LongSum other0 = (LongSum)other;
+
+ if (other0.empty)
+ return;
+
+ empty = false;
+ sum += other0.sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return empty ? null : sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true);
+ }
+ }
+
+ /** */
+ private static class DecimalSum implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = DecimalSum::new;
+
+ /** */
+ private BigDecimal sum;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ BigDecimal in = (BigDecimal)args[0];
+
+ if (in == null)
+ return;
+
+ sum = sum == null ? in : sum.add(in);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DecimalSum other0 = (DecimalSum)other;
+
+ if (other0.sum == null)
+ return;
+
+ sum = sum == null ? other0.sum : sum.add(other0.sum);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true);
+ }
+ }
+
+ /** */
+ private static class DoubleSumEmptyIsZero implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = DoubleSumEmptyIsZero::new;
+
+ /** */
+ private double sum;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Double in = (Double)args[0];
+
+ if (in == null)
+ return;
+
+ sum += in;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DoubleSumEmptyIsZero other0 = (DoubleSumEmptyIsZero)other;
+
+ sum += other0.sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true);
+ }
+ }
+
+ /** */
+ private static class IntSumEmptyIsZero implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = IntSumEmptyIsZero::new;
+
+ /** */
+ private int sum;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Integer in = (Integer)args[0];
+
+ if (in == null)
+ return;
+
+ sum += in;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ IntSumEmptyIsZero other0 = (IntSumEmptyIsZero)other;
+
+ sum += other0.sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true);
+ }
+ }
+
+ /** */
+ private static class LongSumEmptyIsZero implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = LongSumEmptyIsZero::new;
+
+ /** */
+ private long sum;
+
+ /** */
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Long in = (Long)args[0];
+
+ if (in == null)
+ return;
+
+ sum += in;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ LongSumEmptyIsZero other0 = (LongSumEmptyIsZero)other;
+
+ sum += other0.sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return sum;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true);
+ }
+ }
+
+ /** */
+ private static class DecimalSumEmptyIsZero implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> FACTORY = DecimalSumEmptyIsZero::new;
+
+ /** */
+ private BigDecimal sum;
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ BigDecimal in = (BigDecimal)args[0];
+
+ if (in == null)
+ return;
+
+ sum = sum == null ? in : sum.add(in);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DecimalSumEmptyIsZero other0 = (DecimalSumEmptyIsZero)other;
+
+ sum = sum == null ? other0.sum : sum.add(other0.sum);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return sum != null ? sum : BigDecimal.ZERO;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true);
+ }
+ }
+
+ /** */
+ private static class DoubleMinMax implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> MIN_FACTORY = () -> new DoubleMinMax(true);
+
+ /** */
+ public static final Supplier<Accumulator> MAX_FACTORY = () -> new DoubleMinMax(false);
+
+ /** */
+ private final boolean min;
+
+ /** */
+ private double val;
+
+ /** */
+ private boolean empty = true;
+
+ /** */
+ private DoubleMinMax(boolean min) {
+ this.min = min;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Double in = (Double)args[0];
+
+ if (in == null)
+ return;
+
+ val = empty ? in : min ? Math.min(val, in) : Math.max(val, in);
+ empty = false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DoubleMinMax other0 = (DoubleMinMax)other;
+
+ if (other0.empty)
+ return;
+
+ val = empty ? other0.val : min ? Math.min(val, other0.val) : Math.max(val, other0.val);
+ empty = false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return empty ? null : val;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DOUBLE), true);
+ }
+ }
+
+ /** */
+ private static class VarCharMinMax implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> MIN_FACTORY = () -> new VarCharMinMax(true);
+
+ /** */
+ public static final Supplier<Accumulator> MAX_FACTORY = () -> new VarCharMinMax(false);
+
+ /** */
+ private final boolean min;
+
+ /** */
+ private CharSequence val;
+
+ /** */
+ private boolean empty = true;
+
+ /** */
+ private VarCharMinMax(boolean min) {
+ this.min = min;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ CharSequence in = (CharSequence)args[0];
+
+ if (in == null)
+ return;
+
+ val = empty ? in : min ?
+ (CharSeqComparator.INSTANCE.compare(val, in) < 0 ? val : in) :
+ (CharSeqComparator.INSTANCE.compare(val, in) < 0 ? in : val);
+
+ empty = false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ VarCharMinMax other0 = (VarCharMinMax)other;
+
+ if (other0.empty)
+ return;
+
+ val = empty ? other0.val : min ?
+ (CharSeqComparator.INSTANCE.compare(val, other0.val) < 0 ? val : other0.val) :
+ (CharSeqComparator.INSTANCE.compare(val, other0.val) < 0 ? other0.val : val);
+
+ empty = false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return empty ? null : val;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(VARCHAR), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(VARCHAR), true);
+ }
+
+ /** */
+ @SuppressWarnings("ComparatorNotSerializable")
+ private static class CharSeqComparator implements Comparator<CharSequence> {
+ /** */
+ private static final CharSeqComparator INSTANCE = new CharSeqComparator();
+
+ /** */
+ @Override public int compare(CharSequence s1, CharSequence s2) {
+ int len = Math.min(s1.length(), s2.length());
+
+ // find the first difference and return
+ for (int i = 0; i < len; i += 1) {
+ int cmp = Character.compare(s1.charAt(i), s2.charAt(i));
+ if (cmp != 0)
+ return cmp;
+ }
+
+ // if there are no differences, then the shorter seq is first
+ return Integer.compare(s1.length(), s2.length());
+ }
+ }
+ }
+
+ /** */
+ private static class IntMinMax implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> MIN_FACTORY = () -> new IntMinMax(true);
+
+ /** */
+ public static final Supplier<Accumulator> MAX_FACTORY = () -> new IntMinMax(false);
+
+ /** */
+ private final boolean min;
+
+ /** */
+ private int val;
+
+ /** */
+ private boolean empty = true;
+
+ /** */
+ private IntMinMax(boolean min) {
+ this.min = min;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Integer in = (Integer)args[0];
+
+ if (in == null)
+ return;
+
+ val = empty ? in : min ? Math.min(val, in) : Math.max(val, in);
+ empty = false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ IntMinMax other0 = (IntMinMax)other;
+
+ if (other0.empty)
+ return;
+
+ val = empty ? other0.val : min ? Math.min(val, other0.val) : Math.max(val, other0.val);
+ empty = false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return empty ? null : val;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(INTEGER), true);
+ }
+ }
+
+ /** */
+ private static class LongMinMax implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> MIN_FACTORY = () -> new LongMinMax(true);
+
+ /** */
+ public static final Supplier<Accumulator> MAX_FACTORY = () -> new LongMinMax(false);
+
+ /** */
+ private final boolean min;
+
+ /** */
+ private long val;
+
+ /** */
+ private boolean empty = true;
+
+ /** */
+ private LongMinMax(boolean min) {
+ this.min = min;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Long in = (Long)args[0];
+
+ if (in == null)
+ return;
+
+ val = empty ? in : min ? Math.min(val, in) : Math.max(val, in);
+ empty = false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ LongMinMax other0 = (LongMinMax)other;
+
+ if (other0.empty)
+ return;
+
+ val = empty ? other0.val : min ? Math.min(val, other0.val) : Math.max(val, other0.val);
+ empty = false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return empty ? null : val;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(BIGINT), true);
+ }
+ }
+
+ /** */
+ private static class DecimalMinMax implements Accumulator {
+ /** */
+ public static final Supplier<Accumulator> MIN_FACTORY = () -> new DecimalMinMax(true);
+
+ /** */
+ public static final Supplier<Accumulator> MAX_FACTORY = () -> new DecimalMinMax(false);
+
+ /** */
+ private final boolean min;
+
+ /** */
+ private BigDecimal val;
+
+ /** */
+ private DecimalMinMax(boolean min) {
+ this.min = min;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ BigDecimal in = (BigDecimal)args[0];
+
+ if (in == null)
+ return;
+
+ val = val == null ? in : min ? val.min(in) : val.max(in);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DecimalMinMax other0 = (DecimalMinMax)other;
+
+ if (other0.val == null)
+ return;
+
+ val = val == null ? other0.val : min ? val.min(other0.val) : val.max(other0.val);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ return val;
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return List.of(typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true));
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return typeFactory.createTypeWithNullability(typeFactory.createSqlType(DECIMAL), true);
+ }
+ }
+
+ /** */
+ private static class DistinctAccumulator implements Accumulator {
+ /** */
+ private final Accumulator acc;
+
+ /** */
+ private final Set<Object> set = new HashSet<>();
+
+ /** */
+ private DistinctAccumulator(Supplier<Accumulator> accSup) {
+ this.acc = accSup.get();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void add(Object... args) {
+ Object in = args[0];
+
+ if (in == null)
+ return;
+
+ set.add(in);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator other) {
+ DistinctAccumulator other0 = (DistinctAccumulator)other;
+
+ set.addAll(other0.set);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ for (Object o : set)
+ acc.add(o);
+
+ return acc.end();
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+ return acc.argumentTypes(typeFactory);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+ return acc.returnType(typeFactory);
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
new file mode 100644
index 0000000..4626623
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AccumulatorsFactory.java
@@ -0,0 +1,330 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+import java.lang.reflect.Modifier;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
+import com.google.common.primitives.Primitives;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.enumerable.EnumUtils;
+import org.apache.calcite.adapter.enumerable.JavaRowFormat;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+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.MethodDeclaration;
+import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.query.calcite.util.TypeUtils.createRowType;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/** */
+public class AccumulatorsFactory<Row> implements Supplier<List<AccumulatorWrapper<Row>>> {
+ /** */
+ private static final LoadingCache<Pair<RelDataType, RelDataType>, Function<Object, Object>> CACHE =
+ CacheBuilder.newBuilder().build(CacheLoader.from(AccumulatorsFactory::cast0));
+
+ /** */
+ public static interface CastFunction extends Function<Object, Object> {
+ @Override Object apply(Object o);
+ }
+
+ /** */
+ private static Function<Object, Object> cast(RelDataType from, RelDataType to) {
+ assert !from.isStruct();
+ assert !to.isStruct();
+
+ return cast(Pair.of(from, to));
+ }
+
+ /** */
+ private static Function<Object, Object> cast(Pair<RelDataType, RelDataType> types) {
+ try {
+ return CACHE.get(types);
+ }
+ catch (ExecutionException e) {
+ throw new IgniteInternalException(e);
+ }
+ }
+
+ /** */
+ private static Function<Object, Object> cast0(Pair<RelDataType, RelDataType> types) {
+ IgniteTypeFactory typeFactory = PlanningContext.empty().typeFactory();
+
+ RelDataType from = types.left;
+ RelDataType to = types.right;
+
+ Class<?> fromType = Primitives.wrap((Class<?>)typeFactory.getJavaClass(from));
+ Class<?> toType = Primitives.wrap((Class<?>)typeFactory.getJavaClass(to));
+
+ if (toType.isAssignableFrom(fromType))
+ return Function.identity();
+
+ if (Void.class == toType)
+ return o -> null;
+
+ return compileCast(typeFactory, from, to);
+ }
+
+ /** */
+ private static Function<Object, Object> compileCast(IgniteTypeFactory typeFactory, RelDataType from,
+ RelDataType to) {
+ RelDataType rowType = createRowType(typeFactory, from);
+ ParameterExpression in_ = Expressions.parameter(Object.class, "in");
+
+ RexToLixTranslator.InputGetter getter =
+ new RexToLixTranslator.InputGetterImpl(
+ ImmutableList.of(
+ Pair.of(EnumUtils.convert(in_, Object.class, typeFactory.getJavaClass(from)),
+ PhysTypeImpl.of(typeFactory, rowType,
+ JavaRowFormat.SCALAR, false))));
+
+ RexBuilder builder = new RexBuilder(typeFactory);
+ RexProgramBuilder programBuilder = new RexProgramBuilder(rowType, builder);
+ RexNode cast = builder.makeCast(to, builder.makeInputRef(from, 0));
+ programBuilder.addProject(cast, null);
+ RexProgram program = programBuilder.getProgram();
+ BlockBuilder list = new BlockBuilder();
+ List<Expression> projects = RexToLixTranslator.translateProjects(program, typeFactory, SqlConformanceEnum.DEFAULT,
+ list, null, DataContext.ROOT, getter, null);
+ list.add(projects.get(0));
+
+ MethodDeclaration decl = Expressions.methodDecl(
+ Modifier.PUBLIC, Object.class, "apply", ImmutableList.of(in_), list.toBlock());
+
+ return Commons.compile(CastFunction.class, Expressions.toString(List.of(decl), "\n", false));
+ }
+
+ /** */
+ private final ExecutionContext<Row> ctx;
+
+ /** */
+ private final AggregateType type;
+
+ /** */
+ private final RelDataType inputRowType;
+
+ /** */
+ private final List<WrapperPrototype> prototypes;
+
+ /** */
+ public AccumulatorsFactory(
+ ExecutionContext<Row> ctx,
+ AggregateType type,
+ List<AggregateCall> aggCalls,
+ RelDataType inputRowType
+ ) {
+ this.ctx = ctx;
+ this.type = type;
+ this.inputRowType = inputRowType;
+
+ prototypes = Commons.transform(aggCalls, WrapperPrototype::new);
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<AccumulatorWrapper<Row>> get() {
+ return Commons.transform(prototypes, WrapperPrototype::get);
+ }
+
+ /** */
+ private final class WrapperPrototype implements Supplier<AccumulatorWrapper<Row>> {
+ /** */
+ private Supplier<Accumulator> accFactory;
+
+ /** */
+ private final AggregateCall call;
+
+ /** */
+ private Function<Object[], Object[]> inAdapter;
+
+ /** */
+ private Function<Object, Object> outAdapter;
+
+ /** */
+ private WrapperPrototype(AggregateCall call) {
+ this.call = call;
+ }
+
+ /** {@inheritDoc} */
+ @Override public AccumulatorWrapper<Row> get() {
+ Accumulator accumulator = accumulator();
+
+ return new AccumulatorWrapperImpl(accumulator, call, inAdapter, outAdapter);
+ }
+
+ /** */
+ @NotNull private Accumulator accumulator() {
+ if (accFactory != null)
+ return accFactory.get();
+
+ // init factory and adapters
+ accFactory = Accumulators.accumulatorFactory(call);
+ Accumulator accumulator = accFactory.get();
+
+ inAdapter = createInAdapter(accumulator);
+ outAdapter = createOutAdapter(accumulator);
+
+ return accumulator;
+ }
+
+ /** */
+ @NotNull private Function<Object[], Object[]> createInAdapter(Accumulator accumulator) {
+ if (type == AggregateType.REDUCE || nullOrEmpty(call.getArgList()))
+ return Function.identity();
+
+ List<RelDataType> inTypes = SqlTypeUtil.projectTypes(inputRowType, call.getArgList());
+ List<RelDataType> outTypes = accumulator.argumentTypes(ctx.getTypeFactory());
+
+ if (call.ignoreNulls())
+ inTypes = Commons.transform(inTypes, this::nonNull);
+
+ List<Function<Object, Object>> casts =
+ Commons.transform(Pair.zip(inTypes, outTypes), AccumulatorsFactory::cast);
+
+ return new Function<Object[], Object[]>() {
+ @Override public Object[] apply(Object[] args) {
+ for (int i = 0; i < args.length; i++)
+ args[i] = casts.get(i).apply(args[i]);
+ return args;
+ }
+ };
+ }
+
+ /** */
+ @NotNull private Function<Object, Object> createOutAdapter(Accumulator accumulator) {
+ if (type == AggregateType.MAP)
+ return Function.identity();
+
+ RelDataType inType = accumulator.returnType(ctx.getTypeFactory());
+ RelDataType outType = call.getType();
+
+ return cast(inType, outType);
+ }
+
+ /** */
+ private RelDataType nonNull(RelDataType type) {
+ return ctx.getTypeFactory().createTypeWithNullability(type, false);
+ }
+ }
+
+ /** */
+ private final class AccumulatorWrapperImpl implements AccumulatorWrapper<Row> {
+ /** */
+ private final Accumulator accumulator;
+
+ /** */
+ private final Function<Object[], Object[]> inAdapter;
+
+ /** */
+ private final Function<Object, Object> outAdapter;
+
+ /** */
+ private final List<Integer> argList;
+
+ /** */
+ private final int filterArg;
+
+ /** */
+ private final boolean ignoreNulls;
+
+ /** */
+ private final RowHandler<Row> handler;
+
+ /** */
+ AccumulatorWrapperImpl(
+ Accumulator accumulator,
+ AggregateCall call,
+ Function<Object[], Object[]> inAdapter,
+ Function<Object, Object> outAdapter
+ ) {
+ this.accumulator = accumulator;
+ this.inAdapter = inAdapter;
+ this.outAdapter = outAdapter;
+
+ argList = call.getArgList();
+ ignoreNulls = call.ignoreNulls();
+ filterArg = call.hasFilter() ? call.filterArg : -1;
+
+ handler = ctx.rowHandler();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void add(Row row) {
+ assert type != AggregateType.REDUCE;
+
+ if (filterArg >= 0 && Boolean.TRUE != handler.get(filterArg, row))
+ return;
+
+ Object[] args = new Object[argList.size()];
+ for (int i = 0; i < argList.size(); i++) {
+ args[i] = handler.get(argList.get(i), row);
+
+ if (ignoreNulls && args[i] == null)
+ return;
+ }
+
+ accumulator.add(inAdapter.apply(args));
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object end() {
+ assert type != AggregateType.MAP;
+
+ return outAdapter.apply(accumulator.end());
+ }
+
+ /** {@inheritDoc} */
+ @Override public void apply(Accumulator accumulator) {
+ assert type == AggregateType.REDUCE;
+
+ this.accumulator.apply(accumulator);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Accumulator accumulator() {
+ assert type == AggregateType.MAP;
+
+ return accumulator;
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AggregateType.java
similarity index 60%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AggregateType.java
index ce7d926..c743e2a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/AggregateType.java
@@ -14,27 +14,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite;
-/** Stubs */
-public class Stubs {
- /** */
- public static int intFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
- /** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+/**
+ *
+ */
+public enum AggregateType {
+ /** Map phase. */
+ MAP,
- /** */
- public static boolean boolFoo(Object... args) {
- return args == null;
- }
+ /** Reduce phase. */
+ REDUCE,
- /** */
- public static String stringFoo(Object... args) {
- return args == null ? "null" : "not null";
- }
+ /** Single phase aggregate. */
+ SINGLE
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java
new file mode 100644
index 0000000..9606754
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/GroupKey.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.exp.agg;
+
+import java.util.Arrays;
+
+import static org.apache.ignite.internal.util.ArrayUtils.OBJECT_EMPTY_ARRAY;
+
+/**
+ *
+ */
+public class GroupKey {
+ /** */
+ public static final GroupKey EMPTY_GRP_KEY = new GroupKey(OBJECT_EMPTY_ARRAY);
+
+ /** */
+ private final Object[] fields;
+
+ /** */
+ public GroupKey(Object[] fields) {
+ this.fields = fields;
+ }
+
+ /** */
+ public Object field(int idx) {
+ return fields[idx];
+ }
+
+ /** */
+ public int fieldsCount() {
+ return fields.length;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || getClass() != o.getClass())
+ return false;
+
+ GroupKey groupKey = (GroupKey) o;
+
+ return Arrays.equals(fields, groupKey.fields);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ return Arrays.hashCode(fields);
+ }
+
+ /** */
+ public static Builder builder(int rowLen) {
+ return new Builder(rowLen);
+ }
+
+ /** */
+ public static class Builder {
+ /** */
+ private final Object[] fields;
+
+ /** */
+ private int idx;
+
+ /** */
+ private Builder(int rowLen) {
+ fields = new Object[rowLen];
+ }
+
+ /** */
+ public Builder add(Object val) {
+ if (idx == fields.length)
+ throw new IndexOutOfBoundsException();
+
+ fields[idx++] = val;
+
+ return this;
+ }
+
+ /** */
+ public GroupKey build() {
+ assert idx == fields.length;
+
+ return new GroupKey(fields);
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
new file mode 100644
index 0000000..9325885
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractNode.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionCancelledException;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.util.Commons;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteLogger;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * Abstract node of execution tree.
+ */
+public abstract class AbstractNode<Row> implements Node<Row> {
+ /** */
+ protected static final int MODIFY_BATCH_SIZE = 100;//IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_BATCH_SIZE", 100);
+
+ /** */
+ protected static final int IO_BATCH_SIZE = 256;//IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IO_BATCH_SIZE", 256);
+
+ /** */
+ protected static final int IO_BATCH_CNT = 4;//IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IO_BATCH_CNT", 4);
+
+ /** */
+ protected final int inBufSize = Commons.IN_BUFFER_SIZE;//IgniteSystemProperties.getInteger("IGNITE_CALCITE_EXEC_IN_BUFFER_SIZE", 2);
+
+ /** */
+ protected final IgniteLogger log = IgniteLogger.forClass(getClass());
+
+ /** for debug purpose */
+ private volatile Thread thread;
+
+ /**
+ * {@link Inbox} node may not have proper context at creation time in case it
+ * creates on first message received from a remote source. This case the context
+ * sets in scope of {@link Inbox#init(ExecutionContext, RelDataType, Collection, Comparator)} method call.
+ */
+ private ExecutionContext<Row> ctx;
+
+ /** */
+ private RelDataType rowType;
+
+ /** */
+ private Downstream<Row> downstream;
+
+ /** */
+ private boolean closed;
+
+ /** */
+ private List<Node<Row>> sources;
+
+ /**
+ * @param ctx Execution context.
+ */
+ protected AbstractNode(ExecutionContext<Row> ctx, RelDataType rowType) {
+ this.ctx = ctx;
+ this.rowType = rowType;
+ }
+
+ /** {@inheritDoc} */
+ @Override public ExecutionContext<Row> context() {
+ return ctx;
+ }
+
+ /** */
+ protected void context(ExecutionContext<Row> ctx) {
+ this.ctx = ctx;
+ }
+
+ /** {@inheritDoc} */
+ @Override public RelDataType rowType() {
+ return rowType;
+ }
+
+ /** */
+ protected void rowType(RelDataType rowType) {
+ this.rowType = rowType;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void register(List<Node<Row>> sources) {
+ this.sources = sources;
+
+ for (int i = 0; i < sources.size(); i++)
+ sources.get(i).onRegister(requestDownstream(i));
+ }
+
+ /** {@inheritDoc} */
+ @Override public List<Node<Row>> sources() {
+ return sources;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ if (isClosed())
+ return;
+
+ closeInternal();
+
+ if (!nullOrEmpty(sources()))
+ sources().forEach(Commons::closeQuiet);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void rewind() {
+ rewindInternal();
+
+ if (!nullOrEmpty(sources()))
+ sources().forEach(Node::rewind);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onRegister(Downstream<Row> downstream) {
+ this.downstream = downstream;
+ }
+
+ /**
+ * Processes given exception.
+ *
+ * @param e Exception.
+ */
+ public void onError(Throwable e) {
+ if (e instanceof ExecutionCancelledException)
+ log.warn("Execution is cancelled.", e);
+ else
+ onErrorInternal(e);
+ }
+
+ /** */
+ protected void closeInternal() {
+ closed = true;
+ }
+
+ /** */
+ protected abstract void rewindInternal();
+
+ /** */
+ protected void onErrorInternal(Throwable e) {
+ Downstream<Row> downstream = downstream();
+
+ assert downstream != null;
+
+ try {
+ downstream.onError(e);
+ }
+ finally {
+ Commons.closeQuiet(this);
+ }
+ }
+
+ /**
+ * @return {@code true} if the subtree is canceled.
+ */
+ protected boolean isClosed() {
+ return closed;
+ }
+
+ /** */
+ protected void checkState() throws Exception {
+ if (context().isCancelled())
+ throw new ExecutionCancelledException();
+ if (Thread.interrupted())
+ throw new IgniteInternalCheckedException("Thread was interrupted.");
+ if (!IgniteUtils.assertionsEnabled())
+ return;
+ if (thread == null)
+ thread = Thread.currentThread();
+ else
+ assert thread == Thread.currentThread();
+ }
+
+ /** */
+ protected abstract Downstream<Row> requestDownstream(int idx);
+
+ /** */
+ @Override public Downstream<Row> downstream() {
+ return downstream;
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractSetOpNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractSetOpNode.java
new file mode 100644
index 0000000..397bf60
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractSetOpNode.java
@@ -0,0 +1,367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.AggregateType;
+import org.apache.ignite.internal.processors.query.calcite.exec.exp.agg.GroupKey;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ * Abstract execution node for set operators (EXCEPT, INTERSECT).
+ */
+public abstract class AbstractSetOpNode<Row> extends AbstractNode<Row> {
+ /** */
+ private final AggregateType type;
+
+ /** */
+ private final Grouping<Row> grouping;
+
+ /** */
+ private int requested;
+
+ /** */
+ private int waiting;
+
+ /** Current source index. */
+ private int curSrcIdx;
+
+ /** */
+ private boolean inLoop;
+
+ /** */
+ protected AbstractSetOpNode(ExecutionContext<Row> ctx, RelDataType rowType, AggregateType type, boolean all,
+ RowFactory<Row> rowFactory, Grouping<Row> grouping) {
+ super(ctx, rowType);
+
+ this.type = type;
+ this.grouping = grouping;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void request(int rowsCnt) throws Exception {
+ assert !nullOrEmpty(sources());
+ assert rowsCnt > 0 && requested == 0;
+ assert waiting <= 0;
+
+ checkState();
+
+ requested = rowsCnt;
+
+ if (waiting == 0)
+ sources().get(curSrcIdx).request(waiting = inBufSize);
+ else if (!inLoop)
+ context().execute(this::flush, this::onError);
+ }
+
+ /** */
+ public void push(Row row, int idx) throws Exception {
+ assert downstream() != null;
+ assert waiting > 0;
+
+ checkState();
+
+ waiting--;
+
+ grouping.add(row, idx);
+
+ if (waiting == 0)
+ sources().get(curSrcIdx).request(waiting = inBufSize);
+ }
+
+ /** */
+ public void end(int idx) throws Exception {
+ assert downstream() != null;
+ assert waiting > 0;
+ assert curSrcIdx == idx;
+
+ checkState();
+
+ grouping.endOfSet(idx);
+
+ if (type == AggregateType.SINGLE && grouping.isEmpty())
+ curSrcIdx = sources().size(); // Skip subsequent sources.
+ else
+ curSrcIdx++;
+
+ if (curSrcIdx >= sources().size()) {
+ waiting = -1;
+
+ flush();
+ }
+ else
+ sources().get(curSrcIdx).request(waiting);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void rewindInternal() {
+ requested = 0;
+ waiting = 0;
+ grouping.groups.clear();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Downstream<Row> requestDownstream(int idx) {
+ return new Downstream<Row>() {
+ @Override public void push(Row row) throws Exception {
+ AbstractSetOpNode.this.push(row, idx);
+ }
+
+ @Override public void end() throws Exception {
+ AbstractSetOpNode.this.end(idx);
+ }
+
+ @Override public void onError(Throwable e) {
+ AbstractSetOpNode.this.onError(e);
+ }
+ };
+ }
+
+ /** */
+ private void flush() throws Exception {
+ if (isClosed())
+ return;
+
+ checkState();
+
+ assert waiting == -1;
+
+ int processed = 0;
+
+ inLoop = true;
+
+ try {
+ if (requested > 0 && !grouping.isEmpty()) {
+ int toSnd = Math.min(requested, inBufSize - processed);
+
+ for (Row row : grouping.getRows(toSnd)) {
+ requested--;
+
+ downstream().push(row);
+
+ processed++;
+ }
+
+ if (processed >= inBufSize && requested > 0) {
+ // Allow others to do their job.
+ context().execute(this::flush, this::onError);
+
+ return;
+ }
+ }
+ }
+ finally {
+ inLoop = false;
+ }
+
+ if (requested > 0) {
+ requested = 0;
+
+ downstream().end();
+ }
+ }
+
+ /** */
+ protected abstract static class Grouping<Row> {
+ /** */
+ protected final Map<GroupKey, int[]> groups = new HashMap<>();
+
+ /** */
+ protected final RowHandler<Row> hnd;
+
+ /** */
+ protected final AggregateType type;
+
+ /** */
+ protected final boolean all;
+
+ /** */
+ protected final RowFactory<Row> rowFactory;
+
+ /** Processed rows count in current set. */
+ protected int rowsCnt = 0;
+
+ /** */
+ protected Grouping(ExecutionContext<Row> ctx, RowFactory<Row> rowFactory, AggregateType type, boolean all) {
+ hnd = ctx.rowHandler();
+ this.type = type;
+ this.all = all;
+ this.rowFactory = rowFactory;
+ }
+
+ /** */
+ private void add(Row row, int setIdx) {
+ if (type == AggregateType.REDUCE) {
+ assert setIdx == 0 : "Unexpected set index: " + setIdx;
+
+ addOnReducer(row);
+ }
+ else if (type == AggregateType.MAP)
+ addOnMapper(row, setIdx);
+ else
+ addOnSingle(row, setIdx);
+
+ rowsCnt++;
+ }
+
+ /**
+ * @param cnt Number of rows.
+ *
+ * @return Actually sent rows number.
+ */
+ private List<Row> getRows(int cnt) {
+ if (nullOrEmpty(groups))
+ return Collections.emptyList();
+ else if (type == AggregateType.MAP)
+ return getOnMapper(cnt);
+ else
+ return getOnSingleOrReducer(cnt);
+ }
+
+ /** */
+ protected GroupKey key(Row row) {
+ int size = hnd.columnCount(row);
+
+ Object[] fields = new Object[size];
+
+ for (int i = 0; i < size; i++)
+ fields[i] = hnd.get(i, row);
+
+ return new GroupKey(fields);
+ }
+
+ /** */
+ protected void endOfSet(int setIdx) {
+ rowsCnt = 0;
+ }
+
+ /** */
+ protected abstract void addOnSingle(Row row, int setIdx);
+
+ /** */
+ protected abstract void addOnMapper(Row row, int setIdx);
+
+ /** */
+ protected void addOnReducer(Row row) {
+ GroupKey grpKey = (GroupKey)hnd.get(0, row);
+ int[] cntrsMap = (int[])hnd.get(1, row);
+
+ int[] cntrs = groups.computeIfAbsent(grpKey, k -> new int[cntrsMap.length]);
+
+ assert cntrs.length == cntrsMap.length;
+
+ for (int i = 0; i < cntrsMap.length; i++)
+ cntrs[i] += cntrsMap[i];
+ }
+
+ /** */
+ protected List<Row> getOnMapper(int cnt) {
+ Iterator<Map.Entry<GroupKey, int[]>> it = groups.entrySet().iterator();
+
+ int amount = Math.min(cnt, groups.size());
+ List<Row> res = new ArrayList<>(amount);
+
+ while (amount > 0 && it.hasNext()) {
+ Map.Entry<GroupKey, int[]> entry = it.next();
+
+ // Skip row if it doesn't affect the final result.
+ if (affectResult(entry.getValue())) {
+ res.add(rowFactory.create(entry.getKey(), entry.getValue()));
+
+ amount--;
+ }
+
+ it.remove();
+ }
+
+ return res;
+ }
+
+ /** */
+ protected List<Row> getOnSingleOrReducer(int cnt) {
+ Iterator<Map.Entry<GroupKey, int[]>> it = groups.entrySet().iterator();
+
+ List<Row> res = new ArrayList<>(cnt);
+
+ while (it.hasNext() && cnt > 0) {
+ Map.Entry<GroupKey, int[]> entry = it.next();
+
+ GroupKey key = entry.getKey();
+
+ Object[] fields = new Object[key.fieldsCount()];
+
+ for (int i = 0; i < fields.length; i++)
+ fields[i] = key.field(i);
+
+ Row row = rowFactory.create(fields);
+
+ int[] cntrs = entry.getValue();
+
+ int availableRows = availableRows(entry.getValue());
+
+ if (availableRows <= cnt) {
+ it.remove();
+
+ if (availableRows == 0)
+ continue;
+
+ cnt -= availableRows;
+ }
+ else {
+ availableRows = cnt;
+
+ decrementAvailableRows(cntrs, availableRows);
+
+ cnt = 0;
+ }
+
+ for (int i = 0; i < availableRows; i++)
+ res.add(row);
+ }
+
+ return res;
+ }
+
+ /**
+ * Return {@code true} if counters affects the final result, or {@code false} if row can be skipped.
+ */
+ protected abstract boolean affectResult(int[] cntrs);
+
+ /** */
+ protected abstract int availableRows(int[] cntrs);
+
+ /** */
+ protected abstract void decrementAvailableRows(int[] cntrs, int amount);
+
+ /** */
+ private boolean isEmpty() {
+ return groups.isEmpty();
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java
new file mode 100644
index 0000000..de71f0d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/CorrelatedNestedLoopJoinNode.java
@@ -0,0 +1,479 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler;
+
+import static org.apache.ignite.internal.util.CollectionUtils.first;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class CorrelatedNestedLoopJoinNode<Row> extends AbstractNode<Row> {
+ /** */
+ private final Predicate<Row> cond;
+
+ /** */
+ private final List<CorrelationId> correlationIds;
+
+ /** */
+ private final JoinRelType joinType;
+
+ /** */
+ private final RowHandler<Row> handler;
+
+ /** */
+ private final int leftInBufferSize;
+
+ /** */
+ private final int rightInBufferSize;
+
+ /** */
+ private final BitSet leftMatched = new BitSet();
+
+ /** */
+ private int requested;
+
+ /** */
+ private int waitingLeft;
+
+ /** */
+ private int waitingRight;
+
+ /** */
+ private List<Row> leftInBuf;
+
+ /** */
+ private List<Row> rightInBuf;
+
+ /** */
+ private int leftIdx;
+
+ /** */
+ private int rightIdx;
+
+ /** */
+ private Row rightEmptyRow;
+
+ /** */
+ private State state = State.INITIAL;
+
+ /** */
+ private enum State {
+ INITIAL, FILLING_LEFT, FILLING_RIGHT, IDLE, IN_LOOP, END
+ }
+
+ /**
+ * @param ctx Execution context.
+ * @param cond Join expression.
+ */
+ public CorrelatedNestedLoopJoinNode(ExecutionContext<Row> ctx, RelDataType rowType, Predicate<Row> cond,
+ Set<CorrelationId> correlationIds, JoinRelType joinType) {
+ super(ctx, rowType);
+
+ assert !nullOrEmpty(correlationIds);
+
+ this.cond = cond;
+ this.correlationIds = new ArrayList<>(correlationIds);
+ this.joinType = joinType;
+
+ leftInBufferSize = correlationIds.size();
+ rightInBufferSize = inBufSize;
+
+ handler = ctx.rowHandler();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void request(int rowsCnt) throws Exception {
+ assert !nullOrEmpty(sources()) && sources().size() == 2;
+ assert rowsCnt > 0 && requested == 0;
+
+ checkState();
+
+ requested = rowsCnt;
+
+ onRequest();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void rewindInternal() {
+ leftInBuf = null;
+ rightInBuf = null;
+
+ leftIdx = 0;
+ rightIdx = 0;
+
+ requested = 0;
+ waitingLeft = 0;
+ waitingRight = 0;
+
+ state = State.INITIAL;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Downstream<Row> requestDownstream(int idx) {
+ if (idx == 0)
+ return new Downstream<Row>() {
+ /** {@inheritDoc} */
+ @Override public void push(Row row) throws Exception {
+ pushLeft(row);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void end() throws Exception {
+ endLeft();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onError(Throwable e) {
+ CorrelatedNestedLoopJoinNode.this.onError(e);
+ }
+ };
+ else if (idx == 1)
+ return new Downstream<Row>() {
+ /** {@inheritDoc} */
+ @Override public void push(Row row) throws Exception {
+ pushRight(row);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void end() throws Exception {
+ endRight();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onError(Throwable e) {
+ CorrelatedNestedLoopJoinNode.this.onError(e);
+ }
+ };
+
+ throw new IndexOutOfBoundsException();
+ }
+
+ /** */
+ private void pushLeft(Row row) throws Exception {
+ assert downstream() != null;
+ assert waitingLeft > 0;
+
+ checkState();
+
+ waitingLeft--;
+
+ if (leftInBuf == null)
+ leftInBuf = new ArrayList<>(leftInBufferSize);
+
+ leftInBuf.add(row);
+
+ onPushLeft();
+ }
+
+ /** */
+ private void pushRight(Row row) throws Exception {
+ assert downstream() != null;
+ assert waitingRight > 0;
+
+ checkState();
+
+ waitingRight--;
+
+ if (rightInBuf == null)
+ rightInBuf = new ArrayList<>(rightInBufferSize);
+
+ rightInBuf.add(row);
+
+ onPushRight();
+ }
+
+ /** */
+ private void endLeft() throws Exception {
+ assert downstream() != null;
+ assert waitingLeft > 0;
+
+ checkState();
+
+ waitingLeft = -1;
+
+ if (leftInBuf == null)
+ leftInBuf = Collections.emptyList();
+
+ onEndLeft();
+ }
+
+ /** */
+ private void endRight() throws Exception {
+ assert downstream() != null;
+ assert waitingRight > 0;
+
+ checkState();
+
+ waitingRight = -1;
+
+ if (rightInBuf == null)
+ rightInBuf = Collections.emptyList();
+
+ onEndRight();
+ }
+
+ /** */
+ private void onRequest() throws Exception {
+ switch (state) {
+ case IN_LOOP:
+ case FILLING_RIGHT:
+ case FILLING_LEFT:
+ break;
+ case INITIAL:
+ assert waitingLeft == 0;
+ assert waitingRight == 0;
+ assert nullOrEmpty(leftInBuf);
+ assert nullOrEmpty(rightInBuf);
+
+ context().execute(() -> {
+ checkState();
+
+ state = State.FILLING_LEFT;
+ leftSource().request(waitingLeft = leftInBufferSize);
+ }, this::onError);
+
+ break;
+ case IDLE:
+ assert rightInBuf != null;
+ assert leftInBuf != null;
+ assert waitingRight == -1 || waitingRight == 0 && rightInBuf.size() == rightInBufferSize;
+ assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+ context().execute(() -> {
+ checkState();
+
+ join();
+ }, this::onError);
+
+ break;
+
+ case END:
+ downstream().end();
+ break;
+
+ default:
+ throw new AssertionError("Unexpected state:" + state);
+ }
+ }
+
+ /** */
+ private void onPushLeft() throws Exception {
+ assert state == State.FILLING_LEFT : "Unexpected state:" + state;
+ assert waitingRight == 0 || waitingRight == -1;
+ assert nullOrEmpty(rightInBuf);
+
+ if (leftInBuf.size() == leftInBufferSize) {
+ assert waitingLeft == 0;
+
+ prepareCorrelations();
+
+ if (waitingRight == -1)
+ rightSource().rewind();
+
+ state = State.FILLING_RIGHT;
+ rightSource().request(waitingRight = rightInBufferSize);
+ }
+ }
+
+ /** */
+ private void onPushRight() throws Exception {
+ assert state == State.FILLING_RIGHT : "Unexpected state:" + state;
+ assert !nullOrEmpty(leftInBuf);
+ assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+ if (rightInBuf.size() == rightInBufferSize) {
+ assert waitingRight == 0;
+
+ state = State.IDLE;
+
+ join();
+ }
+ }
+
+ /** */
+ private void onEndLeft() throws Exception {
+ assert state == State.FILLING_LEFT : "Unexpected state:" + state;
+ assert waitingLeft == -1;
+ assert waitingRight == 0 || waitingRight == -1;
+ assert nullOrEmpty(rightInBuf);
+
+ if (nullOrEmpty(leftInBuf)) {
+ waitingRight = -1;
+
+ state = State.END;
+
+ if (requested > 0)
+ downstream().end();
+ }
+ else {
+ prepareCorrelations();
+
+ if (waitingRight == -1)
+ rightSource().rewind();
+
+ state = State.FILLING_RIGHT;
+
+ rightSource().request(waitingRight = rightInBufferSize);
+ }
+ }
+
+ /** */
+ private void onEndRight() throws Exception {
+ assert state == State.FILLING_RIGHT : "Unexpected state:" + state;
+ assert waitingRight == -1;
+ assert !nullOrEmpty(leftInBuf);
+ assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;
+
+ state = State.IDLE;
+
+ join();
+ }
+
+ /** */
+ private void join() throws Exception {
+ assert state == State.IDLE;
+
+ state = State.IN_LOOP;
+
+ try {
+ while (requested > 0 && rightIdx < rightInBuf.size()) {
+ if (leftIdx == leftInBuf.size())
+ leftIdx = 0;
+
+ while (requested > 0 && leftIdx < leftInBuf.size()) {
+ checkState();
+
+ Row row = handler.concat(leftInBuf.get(leftIdx), rightInBuf.get(rightIdx));
+
+ if (cond.test(row)) {
+ leftMatched.set(leftIdx);
+
+ requested--;
+
+ downstream().push(row);
+ }
+
+ leftIdx++;
+ }
+
+ if (leftIdx == leftInBuf.size())
+ rightInBuf.set(rightIdx++, null);
+ }
+ }
+ finally {
+ state = State.IDLE;
+ }
+
+ if (rightIdx == rightInBuf.size()) {
+ leftIdx = 0;
+ rightIdx = 0;
+
+ if (waitingRight == 0) {
+ rightInBuf = null;
+
+ state = State.FILLING_RIGHT;
+
+ rightSource().request(waitingRight = rightInBufferSize);
+
+ return;
+ }
+
+ if (joinType == JoinRelType.LEFT && !nullOrEmpty(leftInBuf)) {
+ if (rightEmptyRow == null)
+ rightEmptyRow = handler.factory(context().getTypeFactory(), rightSource().rowType()).create();
+
+ int notMatchedIdx = leftMatched.nextClearBit(0);
+
+ while (requested > 0 && notMatchedIdx < leftInBuf.size()) {
+ downstream().push(handler.concat(leftInBuf.get(notMatchedIdx), rightEmptyRow));
+
+ requested--;
+
+ leftMatched.set(notMatchedIdx);
+
+ notMatchedIdx = leftMatched.nextClearBit(notMatchedIdx + 1);
+ }
+
+ if (requested == 0 && notMatchedIdx < leftInBuf.size())
+ return; // Some rows required to be pushed, wait for request.
+ }
+
+ if (waitingLeft == 0) {
+ rightInBuf = null;
+ leftInBuf = null;
+ leftMatched.clear();
+
+ state = State.FILLING_LEFT;
+
+ leftSource().request(waitingLeft = leftInBufferSize);
+
+ return;
+ }
+
+ assert waitingLeft == -1 && waitingRight == -1;
+
+ if (requested > 0) {
+ leftInBuf = null;
+ rightInBuf = null;
+
+ state = State.END;
+
+ if (requested > 0)
+ downstream().end();
+
+ return;
+ }
+
+ // let's free the rows for GC
+ leftInBuf = Collections.emptyList();
+ rightInBuf = Collections.emptyList();
+ }
+ }
+
+ /** */
+ private Node<Row> leftSource() {
+ return sources().get(0);
+ }
+
+ /** */
+ private Node<Row> rightSource() {
+ return sources().get(1);
+ }
+
+ /** */
+ private void prepareCorrelations() {
+ for (int i = 0; i < correlationIds.size(); i++) {
+ Row row = i < leftInBuf.size() ? leftInBuf.get(i) : first(leftInBuf);
+ context().setCorrelated(row, correlationIds.get(i).getId());
+ }
+ }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
similarity index 53%
copy from modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
copy to modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
index ce7d926..8a1bcc8 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/Stubs.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/Downstream.java
@@ -14,27 +14,32 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.ignite.internal.processors.query.calcite;
-/** Stubs */
-public class Stubs {
- /** */
- public static int intFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+package org.apache.ignite.internal.processors.query.calcite.exec.rel;
- /** */
- public static long longFoo(Object... args) {
- return args == null ? 0 : args.length;
- }
+/**
+ * Represents an abstract data consumer.
+ *
+ * <b>Note</b>: except several cases (like consumer node and mailboxes), {@link Node#request(int)},
+ * {@link Downstream#push(Object)} and {@link Downstream#end()} methods should be used from one single thread.
+ */
+public interface Downstream<Row> {
+ /**
+ * Pushes a row to consumer.
+ *
+ * @param row Data row.
+ */
+ void push(Row row) throws Exception;
- /** */
- public static boolean boolFoo(Object... args) {
- return args == null;
- }
+ /**
+ * Signals that data is over.
+ */
+ void end() throws Exception;
- /** */
- public static String stringFoo(Object... args) {
- return args == null ? "null" : "not null";
- }
+ /**
+ * Notifies consumer about error.
+ *
+ * @param e Error to notify consumer about.
+ */
+ void onError(Throwable e);
}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
new file mode 100644
index 0000000..f8afce8
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/FilterNode.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ignite.internal.processors.query.calcite.exec.rel;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.function.Predicate;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext;
+
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+/**
+ *
+ */
+public class FilterNode<Row> extends AbstractNode<Row> implements SingleNode<Row>, Downstream<Row> {
+ /** */
+ private final Predicate<Row> pred;
+
+ /** */
+ private final Deque<Row> inBuf = new ArrayDeque<>(inBufSize);
+
+ /** */
+ private int requested;
+
+ /** */
+ private int waiting;
+
+ /** */
+ private boolean inLoop;
+
+ /**
+ * @param ctx Execution context.
+ * @param pred Predicate.
+ */
+ public FilterNode(ExecutionContext<Row> ctx, RelDataType rowType, Predicate<Row> pred) {
+ super(ctx, rowType);
+
+ this.pred = pred;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void request(int rowsCnt) throws Exception {
+ assert !nullOrEmpty(sources()) && sources().size() == 1;
+ assert rowsCnt > 0 && requested == 0;
+
+ checkState();
+
+ requested = rowsCnt;
+
+ if (!inLoop)
+ context().execute(this::doFilter, this::onError);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void push(Row row) throws Exception {
+ assert downstream() != null;
+ assert waiting > 0;
+
+ checkState();
+
+ waiting--;
+
+ if (pred.test(row))
+ inBuf.add(row);
+
+ filter();
+ }
+
+ /** {@inheritDoc} */
+ @Override public void end() throws Exception {
+ assert downstream() != null;
+ assert waiting > 0;
+
+ checkState();
+
+ waiting = -1;
+
+ filter();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Downstream<Row> requestDownstream(int idx) {
+ if (idx != 0)
+ throw new IndexOutOfBoundsException();
+
+ return this;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void rewindInternal() {
+ requested = 0;
+ waiting = 0;
+ inBuf.clear();
+ }
+
+ /** */
+ private void doFilter() throws Exception {
+ checkState();
+
+ filter();
+ }
+
+ /** */
+ private void filter() throws Exception {
+ inLoop = true;
+ try {
+ while (requested > 0 && !inBuf.isEmpty()) {
+ checkState();
+
+ requested--;
+ downstream().push(inBuf.remove());
+ }
+ }
+ finally {
+ inLoop = false;
+ }
+
+ if (inBuf.isEmpty() && waiting == 0)
+ source().request(waiting = inBufSize);
+
+ if (waiting == -1 && requested > 0) {
+ assert inBuf.isEmpty();
... 9610 lines suppressed ...