Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-16 Thread via GitHub


ygerzhedovich merged PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045


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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-15 Thread via GitHub


korlov42 commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1677725265


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/InsertSourcesCoercionTest.java:
##
@@ -0,0 +1,665 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.engine.planner.datatypes;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.calcite.rex.RexNode;
+import 
org.apache.ignite.internal.sql.engine.planner.datatypes.utils.NumericPair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.TypePair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.Types;
+import org.apache.ignite.internal.sql.engine.rel.IgniteKeyValueModify;
+import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
+import org.apache.ignite.internal.sql.engine.util.SqlTestUtils;
+import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * A set of test to verify behavior of type coercion for INSERT operations, 
when values belongs to the NUMERIC type family.
+ *
+ * This tests aim to help to understand in which cases implicit cast will 
be added to which values.
+ */
+public class InsertSourcesCoercionTest extends BaseTypeCoercionTest {

Review Comment:
   you forgot to update `MergeSourcesCoercionTest`



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-10 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1672169302


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/UpdateSourcesCoercionTest.java:
##
@@ -0,0 +1,687 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.engine.planner.datatypes;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Stream;
+import org.apache.calcite.rex.RexNode;
+import 
org.apache.ignite.internal.sql.engine.planner.datatypes.utils.NumericPair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.TypePair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.Types;
+import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.rel.IgniteTableModify;
+import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
+import org.apache.ignite.internal.sql.engine.util.SqlTestUtils;
+import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * A set of test to verify behavior of type coercion for UPDATE operations, 
when values belongs to the NUMERIC type family.
+ *
+ * This tests aim to help to understand in which cases implicit cast will 
be added to which values.
+ */
+public class UpdateSourcesCoercionTest extends BaseTypeCoercionTest {
+
+@ParameterizedTest
+@MethodSource("args1")
+public void update1(

Review Comment:
   done



##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/UpdateSourcesCoercionTest.java:
##
@@ -0,0 +1,687 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.engine.planner.datatypes;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Stream;
+import org.apache.calcite.rex.RexNode;
+import 
org.apache.ignite.internal.sql.engine.planner.datatypes.utils.NumericPair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.TypePair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.Types;
+import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.rel.IgniteTableModify;
+import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
+import org.apache.ignite.internal.sql.engine.util.SqlTestUtils;
+import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * A set of test to verify behavior of type coercion for UPDATE operations, 

Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-10 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1672168983


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/InsertSourcesCoercionTest.java:
##
@@ -0,0 +1,665 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.engine.planner.datatypes;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.calcite.rex.RexNode;
+import 
org.apache.ignite.internal.sql.engine.planner.datatypes.utils.NumericPair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.TypePair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.Types;
+import org.apache.ignite.internal.sql.engine.rel.IgniteKeyValueModify;
+import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
+import org.apache.ignite.internal.sql.engine.util.SqlTestUtils;
+import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * A set of test to verify behavior of type coercion for INSERT operations, 
when values belongs to the NUMERIC type family.
+ *
+ * This tests aim to help to understand in which cases implicit cast will 
be added to which values.
+ */
+public class InsertSourcesCoercionTest extends BaseTypeCoercionTest {

Review Comment:
   doen



##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/InsertSourcesCoercionTest.java:
##
@@ -0,0 +1,665 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.engine.planner.datatypes;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.calcite.rex.RexNode;
+import 
org.apache.ignite.internal.sql.engine.planner.datatypes.utils.NumericPair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.TypePair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.Types;
+import org.apache.ignite.internal.sql.engine.rel.IgniteKeyValueModify;
+import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
+import org.apache.ignite.internal.sql.engine.util.SqlTestUtils;
+import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * A set of test to verify behavior of type coercion for INSERT operations, 
when values belongs to the NUMERIC type family.
+ *
+ * This tests aim to help to understand in which cases implicit cast will 

Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-10 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1672168676


##
modules/sql-engine/src/testFixtures/java/org/apache/ignite/internal/sql/engine/util/SqlTestUtils.java:
##
@@ -351,4 +355,50 @@ public static RexNode 
generateLiteralOrValueExpr(ColumnType type, Object value)
 throw new IllegalArgumentException("Unexpected type: " + type);
 }
 }
+
+/** Convert {@link ColumnType} to {@link SqlTypeName}. */
+public static SqlTypeName columnType2SqlTypeName(ColumnType columnType) {
+switch (columnType) {
+case NULL:
+return SqlTypeName.NULL;
+case BOOLEAN:
+return SqlTypeName.BOOLEAN;
+case INT8:
+return SqlTypeName.TINYINT;
+case INT16:
+return SqlTypeName.SMALLINT;
+case INT32:
+return SqlTypeName.INTEGER;
+case INT64:
+return SqlTypeName.BIGINT;
+case FLOAT:
+return SqlTypeName.REAL;
+case DOUBLE:
+return SqlTypeName.DOUBLE;
+case DECIMAL:
+case NUMBER:
+return SqlTypeName.DECIMAL;
+case DATE:
+return SqlTypeName.DATE;
+case TIME:
+return SqlTypeName.TIME;
+case DATETIME:
+return SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE;
+case TIMESTAMP:
+return SqlTypeName.TIMESTAMP;

Review Comment:
   fixed



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-10 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1672136825


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/BaseTypeCoercionTest.java:
##
@@ -51,6 +53,14 @@ static Stream allNumericPairs() {
 return Arrays.stream(NumericPair.values()).map(Arguments::of);
 }
 
+static void checkIncludesAllTypePairs(Stream args) {

Review Comment:
   I will just rename the method



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-10 Thread via GitHub


korlov42 commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1672098248


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -675,7 +676,16 @@ protected void checkSplitAndSerialization(IgniteRel rel, 
IgniteSchema publicSche
 checkSplitAndSerialization(rel, Collections.singleton(publicSchema));
 }
 
+// Set of Relational operators that do not support serialization and 
shouldn't be sent between cluster nodes.
+private static final Set unsupportSerializationOperators = Set.of(
+IgniteKeyValueModify.class

Review Comment:
   IgniteKeyValueGet as well



##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/BaseTypeCoercionTest.java:
##
@@ -51,6 +53,14 @@ static Stream allNumericPairs() {
 return Arrays.stream(NumericPair.values()).map(Arguments::of);
 }
 
+static void checkIncludesAllTypePairs(Stream args) {

Review Comment:
   it's better to revert this change. BaseTypeCoercionTest is base class for 
all type pairs, not only numeric ones.



##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/UpdateSourcesCoercionTest.java:
##
@@ -0,0 +1,687 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.engine.planner.datatypes;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Stream;
+import org.apache.calcite.rex.RexNode;
+import 
org.apache.ignite.internal.sql.engine.planner.datatypes.utils.NumericPair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.TypePair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.Types;
+import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.rel.IgniteTableModify;
+import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
+import org.apache.ignite.internal.sql.engine.util.SqlTestUtils;
+import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * A set of test to verify behavior of type coercion for UPDATE operations, 
when values belongs to the NUMERIC type family.
+ *
+ * This tests aim to help to understand in which cases implicit cast will 
be added to which values.
+ */
+public class UpdateSourcesCoercionTest extends BaseTypeCoercionTest {
+
+@ParameterizedTest
+@MethodSource("args1")
+public void update1(
+TypePair pair,
+Matcher operandMatcher
+) throws Exception {
+// TODO: remove during implement IGNITE-22283
+if (pair.first().spec() == NativeTypeSpec.NUMBER || 
pair.second().spec() == NativeTypeSpec.NUMBER) {
+return;
+}
+
+IgniteSchema schema = createSchemaWithTwoColumnTable(pair.first(), 
pair.second());
+
+Object val = 
SqlTestUtils.generateValueByType(pair.second().spec().asColumnType());
+
+assertPlan("UPDATE T SET c1=" + val, schema, 
modifyOperandMatcher(operandMatcher)::matches, List.of());
+}
+
+@ParameterizedTest
+@MethodSource("argsDyn")
+public void updateDynamicParameters(
+TypePair pair,
+Matcher operandMatcher
+) throws Exception {
+// TODO: remove during implement IGNITE-22283
+if (pair.first().spec() == NativeTypeSpec.NUMBER || 
pair.second().spec() == NativeTypeSpec.NUMBER) {
+return;
+}
+
+IgniteSchema schema = createSchemaWithTwoColumnTable(pair.first(), 
pair.second());
+
+Object val = 
SqlTestUtils.generateValueByType(pair.second().spec().asColumnType());
+
+assertPlan("UPDATE T SET c1=?", schema, 

Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1670247930


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   reworked a bit



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r166993


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/InsertSourcesCoercionTest.java:
##
@@ -0,0 +1,665 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.engine.planner.datatypes;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.calcite.rex.RexNode;
+import 
org.apache.ignite.internal.sql.engine.planner.datatypes.utils.NumericPair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.TypePair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.Types;
+import org.apache.ignite.internal.sql.engine.rel.IgniteKeyValueModify;
+import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
+import org.apache.ignite.internal.sql.engine.util.SqlTestUtils;
+import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * A set of test to verify behavior of type coercion for INSERT operations, 
when values belongs to the NUMERIC type family.
+ *
+ * This tests aim to help to understand in which cases implicit cast will 
be added to which values.
+ */
+public class InsertSourcesCoercionTest extends BaseTypeCoercionTest {
+
+@ParameterizedTest
+@MethodSource("args")
+public void insert(
+TypePair pair,
+Matcher operandMatcher
+) throws Exception {
+
+IgniteSchema schema = createSchemaWithTwoColumnTable(pair.first(), 
pair.first());
+
+Object val = 
SqlTestUtils.generateValueByType(pair.second().spec().asColumnType());
+
+assertPlan("INSERT INTO T VALUES(" + val + "," + val + ")", schema, 
keyValOperandMatcher(operandMatcher)::matches, List.of());
+}
+
+@ParameterizedTest
+@MethodSource("argsDyn")
+public void insertDynamicParameters(
+TypePair pair,
+Matcher operandMatcher
+) throws Exception {
+// ToDo: remove during implement IGNITE-22283

Review Comment:
   Always use the same approach. But anyway - amended to ``TODO``



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669916020


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   partially covered by our integration tests. If we need wider cover it should 
be separated tests, not as part of planner tests



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669912429


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   This check greatly simplifies detection bugs like this one 
https://issues.apache.org/jira/browse/IGNITE-19858 or this 
https://issues.apache.org/jira/browse/IGNITE-20170.
   



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669912429


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   This check greatly simplifies detection bugs like this one 
https://issues.apache.org/jira/browse/IGNITE-19858
   



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669899583


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   KeyValue plans are not serialisable, but it does not mean that all 
serialisation should be disabled.



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669893119


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/datatypes/InsertSourcesCoercionTest.java:
##
@@ -0,0 +1,665 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.sql.engine.planner.datatypes;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.calcite.rex.RexNode;
+import 
org.apache.ignite.internal.sql.engine.planner.datatypes.utils.NumericPair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.TypePair;
+import org.apache.ignite.internal.sql.engine.planner.datatypes.utils.Types;
+import org.apache.ignite.internal.sql.engine.rel.IgniteKeyValueModify;
+import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
+import org.apache.ignite.internal.sql.engine.util.SqlTestUtils;
+import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * A set of test to verify behavior of type coercion for INSERT operations, 
when values belongs to the NUMERIC type family.
+ *
+ * This tests aim to help to understand in which cases implicit cast will 
be added to which values.
+ */
+public class InsertSourcesCoercionTest extends BaseTypeCoercionTest {
+
+@ParameterizedTest
+@MethodSource("args")
+public void insert(
+TypePair pair,
+Matcher operandMatcher
+) throws Exception {
+
+IgniteSchema schema = createSchemaWithTwoColumnTable(pair.first(), 
pair.first());
+
+Object val = 
SqlTestUtils.generateValueByType(pair.second().spec().asColumnType());
+
+assertPlan("INSERT INTO T VALUES(" + val + "," + val + ")", schema, 
keyValOperandMatcher(operandMatcher)::matches, List.of());
+}
+
+@ParameterizedTest
+@MethodSource("argsDyn")
+public void insertDynamicParameters(
+TypePair pair,
+Matcher operandMatcher
+) throws Exception {
+// ToDo: remove during implement IGNITE-22283

Review Comment:
   Do we really what to introduce another style of writing `TODO` ?



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669883521


##
modules/sql-engine/src/testFixtures/java/org/apache/ignite/internal/sql/engine/util/SqlTestUtils.java:
##
@@ -351,4 +355,50 @@ public static RexNode 
generateLiteralOrValueExpr(ColumnType type, Object value)
 throw new IllegalArgumentException("Unexpected type: " + type);
 }
 }
+
+/** Convert {@link ColumnType} to {@link SqlTypeName}. */
+public static SqlTypeName columnType2SqlTypeName(ColumnType columnType) {
+switch (columnType) {
+case NULL:
+return SqlTypeName.NULL;
+case BOOLEAN:
+return SqlTypeName.BOOLEAN;
+case INT8:
+return SqlTypeName.TINYINT;
+case INT16:
+return SqlTypeName.SMALLINT;
+case INT32:
+return SqlTypeName.INTEGER;
+case INT64:
+return SqlTypeName.BIGINT;
+case FLOAT:
+return SqlTypeName.REAL;
+case DOUBLE:
+return SqlTypeName.DOUBLE;
+case DECIMAL:
+case NUMBER:
+return SqlTypeName.DECIMAL;
+case DATE:
+return SqlTypeName.DATE;
+case TIME:
+return SqlTypeName.TIME;
+case DATETIME:
+return SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE;
+case TIMESTAMP:
+return SqlTypeName.TIMESTAMP;
+case UUID:
+case BITMASK:
+return SqlTypeName.OTHER;

Review Comment:
   fixed for UUID. For BITMASK it doesn't matter, we don't support the type and 
it should be get rid under IGNITE-22283



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669878184


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   How are going to ensure that a plan can be sent (and it is sent/read w/o 
lost of data) to another node?



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669878184


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   How are going to ensure that a plan can be sent to another node?



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669873741


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPruningPredicateSelfTest.java:
##
@@ -118,12 +120,20 @@ public void testLiteralValue(ColumnType columnType) {
 expectPartitionsPruned(table, columns, new Object[0], group, val);
 }
 
+private static NativeType getNativeType(ColumnType columnType) {
+SqlTypeName sqlTypeName = 
SqlTestUtils.columnType2SqlTypeName(columnType);
+int precision = IgniteTypeSystem.INSTANCE.getMaxPrecision(sqlTypeName);
+int scale = IgniteTypeSystem.INSTANCE.getMaxScale(sqlTypeName);
+
+return TypeUtils.columnType2NativeType(columnType, precision, scale, 
4);

Review Comment:
   agree. Fixed



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


ygerzhedovich commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669871584


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   planner tests shouldn't check serialization



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669846842


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java:
##
@@ -499,8 +499,6 @@ protected  void assertPlan(
 String planString = RelOptUtil.dumpPlan("", plan, 
SqlExplainFormat.TEXT, DEFAULT_EXPLAIN_LEVEL);
 log.info("statement: {}\n{}", sql, planString);
 
-checkSplitAndSerialization(plan, schemas);

Review Comment:
   I would rather introduce an option to disable. this check with a default 
value that tells to always check.



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669844300


##
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/pruning/PartitionPruningPredicateSelfTest.java:
##
@@ -118,12 +120,20 @@ public void testLiteralValue(ColumnType columnType) {
 expectPartitionsPruned(table, columns, new Object[0], group, val);
 }
 
+private static NativeType getNativeType(ColumnType columnType) {
+SqlTypeName sqlTypeName = 
SqlTestUtils.columnType2SqlTypeName(columnType);
+int precision = IgniteTypeSystem.INSTANCE.getMaxPrecision(sqlTypeName);
+int scale = IgniteTypeSystem.INSTANCE.getMaxScale(sqlTypeName);
+
+return TypeUtils.columnType2NativeType(columnType, precision, scale, 
4);

Review Comment:
   The length parameter should have the same value as the precision (because 
they are the same thing). 
   



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22581 Add planner test to verify numeric type coercion of source for INSERT, UPDATE and MERGE operators [ignite-3]

2024-07-09 Thread via GitHub


lowka commented on code in PR #4045:
URL: https://github.com/apache/ignite-3/pull/4045#discussion_r1669842572


##
modules/sql-engine/src/testFixtures/java/org/apache/ignite/internal/sql/engine/util/SqlTestUtils.java:
##
@@ -351,4 +355,50 @@ public static RexNode 
generateLiteralOrValueExpr(ColumnType type, Object value)
 throw new IllegalArgumentException("Unexpected type: " + type);
 }
 }
+
+/** Convert {@link ColumnType} to {@link SqlTypeName}. */
+public static SqlTypeName columnType2SqlTypeName(ColumnType columnType) {
+switch (columnType) {
+case NULL:
+return SqlTypeName.NULL;
+case BOOLEAN:
+return SqlTypeName.BOOLEAN;
+case INT8:
+return SqlTypeName.TINYINT;
+case INT16:
+return SqlTypeName.SMALLINT;
+case INT32:
+return SqlTypeName.INTEGER;
+case INT64:
+return SqlTypeName.BIGINT;
+case FLOAT:
+return SqlTypeName.REAL;
+case DOUBLE:
+return SqlTypeName.DOUBLE;
+case DECIMAL:
+case NUMBER:
+return SqlTypeName.DECIMAL;
+case DATE:
+return SqlTypeName.DATE;
+case TIME:
+return SqlTypeName.TIME;
+case DATETIME:
+return SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE;
+case TIMESTAMP:
+return SqlTypeName.TIMESTAMP;
+case UUID:
+case BITMASK:
+return SqlTypeName.OTHER;

Review Comment:
   There is no type that uses SqlTypeName.OTHER. UUID uses ANY, and there is no 
SqlType for BITMASK.



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

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

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