You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2023/08/21 10:54:22 UTC

[ignite] branch master updated: IGNITE-20253 SQL Calcite: Replace placeholder values from ModifyNode (backport of IGNITE-19096) (#10906)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 8f2bbede0ed IGNITE-20253 SQL Calcite: Replace placeholder values from ModifyNode (backport of IGNITE-19096) (#10906)
8f2bbede0ed is described below

commit 8f2bbede0edb2a87502790fe8b09e547945e1937
Author: Ivan Daschinskiy <iv...@apache.org>
AuthorDate: Mon Aug 21 13:54:10 2023 +0300

    IGNITE-20253 SQL Calcite: Replace placeholder values from ModifyNode (backport of IGNITE-19096) (#10906)
    
    Co-authored-by: zstan <st...@gmail.com>
---
 .../query/calcite/exec/exp/RexImpTable.java        |   7 +-
 .../calcite/prepare/IgniteSqlToRelConvertor.java   | 129 +++++++++++++++++++++
 .../calcite/schema/CacheTableDescriptorImpl.java   |  20 +---
 .../integration/TableDmlIntegrationTest.java       |  37 ++++++
 4 files changed, 172 insertions(+), 21 deletions(-)

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
index 366dcc199e3..0ec1dfb6ba6 100644
--- 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
@@ -260,9 +260,6 @@ public class RexImpTable {
     /** */
     private final Map<SqlOperator, RexCallImplementor> map = new HashMap<>();
 
-    /** Placeholder for DEFAULT operator value. */
-    public static final Object DEFAULT_VALUE_PLACEHOLDER = new Object();
-
     /** */
     RexImpTable() {
         defineMethod(ROW, BuiltInMethod.ARRAY.method, NullPolicy.NONE);
@@ -2513,7 +2510,7 @@ public class RexImpTable {
     private static class DefaultImplementor extends AbstractRexCallImplementor {
         /** */
         DefaultImplementor() {
-            super(NullPolicy.NONE, false);
+            super(NullPolicy.NONE, true);
         }
 
         /** {@inheritDoc} */
@@ -2524,7 +2521,7 @@ public class RexImpTable {
         /** {@inheritDoc} */
         @Override Expression implementSafe(final RexToLixTranslator translator,
             final RexCall call, final List<Expression> argValueList) {
-            return Expressions.field(null, RexImpTable.class, "DEFAULT_VALUE_PLACEHOLDER");
+            return Expressions.constant(null);
         }
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlToRelConvertor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlToRelConvertor.java
index eda8011066e..d72729ab3a6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlToRelConvertor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlToRelConvertor.java
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
+import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Deque;
 import java.util.List;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
@@ -28,21 +30,28 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlInsert;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlMerge;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlUpdate;
+import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.Pair;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 import static java.util.Objects.requireNonNull;
@@ -51,6 +60,12 @@ import static java.util.Objects.requireNonNull;
  * Converts a SQL parse tree into a relational algebra operators.
  */
 public class IgniteSqlToRelConvertor extends SqlToRelConverter {
+    /** */
+    private final Deque<SqlCall> datasetStack = new ArrayDeque<>();
+
+    /** */
+    private RelBuilder relBuilder;
+
     /** */
     public IgniteSqlToRelConvertor(
         RelOptTable.ViewExpander viewExpander,
@@ -60,6 +75,8 @@ public class IgniteSqlToRelConvertor extends SqlToRelConverter {
         Config cfg
     ) {
         super(viewExpander, validator, catalogReader, cluster, convertletTable, cfg);
+
+        relBuilder = config.getRelBuilderFactory().create(cluster, null);
     }
 
     /** {@inheritDoc} */
@@ -70,6 +87,118 @@ public class IgniteSqlToRelConvertor extends SqlToRelConverter {
             return super.convertQueryRecursive(qry, top, targetRowType);
     }
 
+    /** {@inheritDoc} */
+    @Override protected RelNode convertInsert(SqlInsert call) {
+        datasetStack.push(call);
+
+        RelNode rel = super.convertInsert(call);
+
+        datasetStack.pop();
+
+        return rel;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RelNode convertValues(SqlCall values, RelDataType targetRowType) {
+        DefaultChecker checker = new DefaultChecker();
+
+        boolean hasDefaults = checker.hasDefaults(values);
+
+        if (hasDefaults) {
+            SqlValidatorScope scope = validator.getOverScope(values);
+            assert scope != null;
+            Blackboard bb = createBlackboard(scope, null, false);
+
+            convertValuesImplEx(bb, values, targetRowType);
+            return bb.root();
+        }
+        else {
+            // a bit lightweight than default processing one.
+            return super.convertValues(values, targetRowType);
+        }
+    }
+
+    /** */
+    private void convertValuesImplEx(Blackboard bb, SqlCall values, RelDataType targetRowType) {
+        SqlCall insertOp = datasetStack.peek();
+
+        assert insertOp instanceof SqlInsert;
+        assert values == ((SqlInsert)insertOp).getSource();
+
+        RelOptTable targetTable = getTargetTable(insertOp);
+
+        assert targetTable != null;
+
+        IgniteTable ignTable = targetTable.unwrap(IgniteTable.class);
+
+        List<RelDataTypeField> tblFields = targetTable.getRowType().getFieldList();
+        List<String> targetFields = targetRowType.getFieldNames();
+
+        int[] mapping = new int[targetFields.size()];
+
+        int pos = 0;
+        for (String fld : targetFields) {
+            int tblPos = 0;
+            for (RelDataTypeField tblFld : tblFields) {
+                if (tblFld.getName().equals(fld)) {
+                    mapping[pos++] = tblPos;
+                    break;
+                }
+                ++tblPos;
+            }
+        }
+
+        for (SqlNode rowConstructor : values.getOperandList()) {
+            SqlCall rowConstructor0 = (SqlCall)rowConstructor;
+
+            List<Pair<RexNode, String>> exps = new ArrayList<>(targetFields.size());
+
+            pos = 0;
+            for (; pos < targetFields.size(); ++pos) {
+                SqlNode operand = rowConstructor0.getOperandList().get(pos);
+
+                if (operand.getKind() == SqlKind.DEFAULT) {
+                    RexNode def = ignTable.descriptor().newColumnDefaultValue(targetTable, mapping[pos], bb);
+
+                    exps.add(Pair.of(def, SqlValidatorUtil.getAlias(operand, pos)));
+                }
+                else {
+                    exps.add(Pair.of(bb.convertExpression(operand), SqlValidatorUtil.getAlias(operand, pos)));
+                }
+            }
+
+            RelNode in = (null == bb.root) ? LogicalValues.createOneRow(cluster) : bb.root;
+
+            relBuilder.push(in).project(Pair.left(exps), Pair.right(exps));
+        }
+
+        bb.setRoot(relBuilder.union(true, values.getOperandList().size()).build(), true);
+    }
+
+    /** */
+    private static class DefaultChecker extends SqlShuttle {
+        /** */
+        private boolean hasDefaults(SqlCall call) {
+            try {
+                call.accept(this);
+
+                return false;
+            }
+            catch (ControlFlowException e) {
+                return true;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public @Nullable SqlNode visit(SqlCall call) {
+            if (call.getKind() == SqlKind.DEFAULT) {
+                throw new ControlFlowException();
+            }
+
+            return super.visit(call);
+        }
+    }
+
     /**
      * This method was copy-pasted from super-method except this changes:
      * - For updateCall we require all columns in the project and should not skip anything.
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableDescriptorImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableDescriptorImpl.java
index 2a64f93962e..e19a5420350 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableDescriptorImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableDescriptorImpl.java
@@ -59,7 +59,6 @@ import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 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.exp.RexImpTable;
 import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup;
 import org.apache.ignite.internal.processors.query.calcite.prepare.BaseDataContext;
 import org.apache.ignite.internal.processors.query.calcite.prepare.MappingQueryContext;
@@ -355,11 +354,8 @@ public class CacheTableDescriptorImpl extends NullInitializerExpressionFactory
 
         Object key = handler.get(keyField, row);
 
-        if (key != null) {
-            key = replaceDefault(key, descriptors[QueryUtils.KEY_COL]);
-
+        if (key != null)
             return TypeUtils.fromInternal(ectx, key, descriptors[QueryUtils.KEY_COL].storageType());
-        }
 
         // skip _key and _val
         for (int i = 2; i < descriptors.length; i++) {
@@ -368,7 +364,7 @@ public class CacheTableDescriptorImpl extends NullInitializerExpressionFactory
             if (!desc.field() || !desc.key())
                 continue;
 
-            Object fieldVal = replaceDefault(handler.get(i, row), desc);
+            Object fieldVal = handler.get(i, row);
 
             if (fieldVal != null) {
                 if (key == null)
@@ -397,26 +393,18 @@ public class CacheTableDescriptorImpl extends NullInitializerExpressionFactory
             for (int i = 2; i < descriptors.length; i++) {
                 final CacheColumnDescriptor desc = descriptors[i];
 
-                Object fieldVal = replaceDefault(handler.get(i, row), desc);
+                Object fieldVal = handler.get(i, row);
 
                 if (desc.field() && !desc.key() && fieldVal != null)
                     desc.set(val, TypeUtils.fromInternal(ectx, fieldVal, desc.storageType()));
             }
         }
-        else {
-            val = replaceDefault(val, descriptors[QueryUtils.VAL_COL]);
-
+        else
             val = TypeUtils.fromInternal(ectx, val, descriptors[QueryUtils.VAL_COL].storageType());
-        }
 
         return val;
     }
 
-    /** */
-    private Object replaceDefault(Object val, ColumnDescriptor desc) {
-        return val == RexImpTable.DEFAULT_VALUE_PLACEHOLDER ? desc.defaultValue() : val;
-    }
-
     /** */
     private Object newVal(String typeName, Class<?> typeCls) throws IgniteCheckedException {
         GridCacheContext<?, ?> cctx = cacheContext();
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDmlIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDmlIntegrationTest.java
index 06f5b58a142..73f4d2d29ae 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDmlIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDmlIntegrationTest.java
@@ -29,6 +29,7 @@ import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -492,6 +493,42 @@ public class TableDmlIntegrationTest extends AbstractBasicIntegrationTest {
             "Object 'NON_EXISTENT_TABLE' not found");
     }
 
+    /** */
+    @Test
+    public void testInsertMultipleDefaults() {
+        Stream.of(true, false).forEach(withPk -> {
+            try {
+                sql("CREATE TABLE integers(i INTEGER " + (withPk ? "PRIMARY KEY" : "") +
+                        " , col1 INTEGER DEFAULT 200, col2 INTEGER DEFAULT 300)");
+
+                sql("INSERT INTO integers (i) VALUES (0)");
+                sql("INSERT INTO integers VALUES (1, DEFAULT, DEFAULT)");
+                sql("INSERT INTO integers(i, col2) VALUES (2, DEFAULT), (3, 4), (4, DEFAULT)");
+                sql("INSERT INTO integers VALUES (5, DEFAULT, DEFAULT)");
+                sql("INSERT INTO integers VALUES (6, 4, DEFAULT)");
+                sql("INSERT INTO integers VALUES (7, 5, 5)");
+                sql("INSERT INTO integers(col1, i) VALUES (DEFAULT, 8)");
+                sql("INSERT INTO integers(i, col1) VALUES (9, DEFAULT)");
+
+                assertQuery("SELECT i, col1, col2 FROM integers ORDER BY i")
+                        .returns(0, 200, 300)
+                        .returns(1, 200, 300)
+                        .returns(2, 200, 300)
+                        .returns(3, 200, 4)
+                        .returns(4, 200, 300)
+                        .returns(5, 200, 300)
+                        .returns(6, 4, 300)
+                        .returns(7, 5, 5)
+                        .returns(8, 200, 300)
+                        .returns(9, 200, 300)
+                        .check();
+            }
+            finally {
+                sql("DROP TABLE IF EXISTS integers");
+            }
+        });
+    }
+
     /** */
     @Test
     public void testInsertDefaultValue() {