snuyanzin commented on code in PR #25316:
URL: https://github.com/apache/flink/pull/25316#discussion_r1829780200


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/FlinkCalciteSqlValidatorTest.java:
##########
@@ -50,74 +67,43 @@ void testUpsertInto() {
                         "UPSERT INTO statement is not supported. Please use 
INSERT INTO instead.");
     }
 
-    @Test
-    void testInsertIntoShouldColumnMismatchWithValues() {
-        assertThatThrownBy(() -> plannerMocks.getParser().parse("INSERT INTO 
t2 (a,b) VALUES(1)"))
-                .isInstanceOf(ValidationException.class)
-                .hasMessageContaining(" Number of columns must match number of 
query columns");
-    }
-
-    @Test
-    void testInsertIntoShouldColumnMismatchWithSelect() {
-        assertThatThrownBy(() -> plannerMocks.getParser().parse("INSERT INTO 
t2 (a,b) SELECT 1"))
-                .isInstanceOf(ValidationException.class)
-                .hasMessageContaining(" Number of columns must match number of 
query columns");
-    }
-
-    @Test
-    void testInsertIntoShouldColumnMismatchWithLastValue() {
-        assertThatThrownBy(
-                        () ->
-                                plannerMocks
-                                        .getParser()
-                                        .parse("INSERT INTO t2 (a,b) VALUES 
(1,2), (3)"))
-                .isInstanceOf(ValidationException.class)
-                .hasMessageContaining(" Number of columns must match number of 
query columns");
-    }
-
-    @Test
-    void testInsertIntoShouldColumnMismatchWithFirstValue() {
-        assertThatThrownBy(
-                        () ->
-                                plannerMocks
-                                        .getParser()
-                                        .parse("INSERT INTO t2 (a,b) VALUES 
(1), (2,3)"))
-                .isInstanceOf(ValidationException.class)
-                .hasMessageContaining(" Number of columns must match number of 
query columns");
-    }
-
-    @Test
-    void testInsertIntoShouldColumnMismatchWithMultiFieldValues() {
-        assertThatThrownBy(
-                        () ->
-                                plannerMocks
-                                        .getParser()
-                                        .parse("INSERT INTO t2 (a,b) VALUES 
(1,2), (3,4,5)"))
+    @ParameterizedTest
+    @ValueSource(
+            strings = {
+                "INSERT INTO t2 (a, b) VALUES(1)",
+                "INSERT INTO t2 (a, b) VALUES (1, 2), (3)",
+                "INSERT INTO t2 (a, b) VALUES (1), (2, 3)",
+                "INSERT INTO t2 (a, b) VALUES (1, 2), (3, 4, 5)",
+                "INSERT INTO t2 (a, b) SELECT 1",
+                "INSERT INTO t2 (a, b) SELECT * FROM t1",
+                "INSERT INTO t2 (a, b) SELECT *, 42 FROM t2_copy",
+                "INSERT INTO t2 (a, b) SELECT 42, * FROM t2_copy",
+                "INSERT INTO t2 (a, b) SELECT * FROM t_nested",
+                "INSERT INTO t2 (a, b) TABLE t_nested",
+                "INSERT INTO t2 (a, b) SELECT * FROM (TABLE t_nested)"
+            })
+    void testInvalidNumberOfColumnsWhileInsertInto(String sql) {
+        assertThatThrownBy(() -> plannerMocks.getParser().parse(sql))
                 .isInstanceOf(ValidationException.class)
                 .hasMessageContaining(" Number of columns must match number of 
query columns");
     }
 
-    @Test
-    void testInsertIntoShouldNotColumnMismatchWithValues() {
-        assertDoesNotThrow(
-                () -> {
-                    plannerMocks.getParser().parse("INSERT INTO t2 (a,b) 
VALUES (1,2), (3,4)");
-                });
-    }
-
-    @Test
-    void testInsertIntoShouldNotColumnMismatchWithSelect() {
-        assertDoesNotThrow(
-                () -> {
-                    plannerMocks.getParser().parse("INSERT INTO t2 (a,b) 
Select 1, 2");
-                });
-    }
-
-    @Test
-    void testInsertIntoShouldNotColumnMismatchWithSingleColValues() {
+    @ParameterizedTest
+    @ValueSource(
+            strings = {
+                "INSERT INTO t2 (a, b) VALUES (1, 2), (3, 4)",
+                "INSERT INTO t2 (a) VALUES (1), (3)",
+                "INSERT INTO t2 (a, b) SELECT 1, 2",
+                "INSERT INTO t2 (a, b) SELECT * FROM t2_copy",
+                "INSERT INTO t2 (a, b) SELECT *, 42 FROM t1",
+                "INSERT INTO t2 (a, b) SELECT 42, * FROM t1",
+                "INSERT INTO t2 (a, b) SELECT f.* FROM t_nested",
+                "INSERT INTO t2 (a, b) TABLE t2_copy"

Review Comment:
   Seems I got what you mean
   from one side there was a wrong condition in filter, fix that, thanks for 
catching
   
   from another side currently there are 2 checks
   one at 
[SqlIdentifier](https://github.com/apache/flink/pull/25316/files#diff-12ede804afb130bfe9c83df9da2ac5745b14345d7a71bee0e803257e6df73951R136)
   
   another at 
https://github.com/apache/flink/pull/25316/files#diff-12ede804afb130bfe9c83df9da2ac5745b14345d7a71bee0e803257e6df73951R187
   
   the difference is that first could be executed immediately (kind of short 
circuit)
   second will be executed only after validation and star expansion...
   
   We can remove the first one however it means that for all queries we will do 
first validation check and then column number check even if there is no star
   
   WDYT?



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to