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

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink-table-store.git


The following commit(s) were added to refs/heads/master by this push:
     new 1840de12 [FLINK-31323] Fix unstable E2E test for flink actions
1840de12 is described below

commit 1840de127c453b710d4aa0e1709c0001880f62f1
Author: yuzelin <[email protected]>
AuthorDate: Tue Mar 7 10:33:21 2023 +0800

    [FLINK-31323] Fix unstable E2E test for flink actions
    
    This closes #580
---
 .../table/store/tests/FileStoreBatchE2eTest.java   |  3 +-
 .../table/store/tests/FlinkActionsE2eTest.java     |  2 -
 .../flink-table-store-flink-1.14/pom.xml           |  8 +++
 .../connector/utils/TableEnvironmentUtils.java     | 66 ++++++++++++++++++++++
 .../table/store/connector/FlinkActionITCase.java   | 63 +++++++++++++++++++++
 .../flink-table-store-flink-1.15/pom.xml           |  8 +++
 .../table/store/connector/FlinkActionITCase.java   | 63 +++++++++++++++++++++
 .../table/store/connector/action/ActionBase.java   | 25 +++++---
 .../table/store/connector/action/DeleteAction.java |  2 +-
 .../store/connector/action/MergeIntoAction.java    |  2 +-
 .../connector/utils/TableEnvironmentUtils.java     | 66 ++++++++++++++++++++++
 11 files changed, 296 insertions(+), 12 deletions(-)

diff --git 
a/flink-table-store-e2e-tests/src/test/java/org/apache/flink/table/store/tests/FileStoreBatchE2eTest.java
 
b/flink-table-store-e2e-tests/src/test/java/org/apache/flink/table/store/tests/FileStoreBatchE2eTest.java
index c2f175ae..3cff1c71 100644
--- 
a/flink-table-store-e2e-tests/src/test/java/org/apache/flink/table/store/tests/FileStoreBatchE2eTest.java
+++ 
b/flink-table-store-e2e-tests/src/test/java/org/apache/flink/table/store/tests/FileStoreBatchE2eTest.java
@@ -162,7 +162,8 @@ public class FileStoreBatchE2eTest extends E2eTestBase {
 
         // test #4: aggregation
         runSql(
-                "INSERT INTO result4 SELECT dt, category, sum(price) AS total 
FROM ts_table GROUP BY dt, category;",
+                "SET 'table.exec.resource.default-parallelism' = '1';\n"
+                        + "INSERT INTO result4 SELECT dt, category, sum(price) 
AS total FROM ts_table GROUP BY dt, category;",
                 catalogDdl,
                 useCatalogCmd,
                 tableStoreDdl,
diff --git 
a/flink-table-store-e2e-tests/src/test/java/org/apache/flink/table/store/tests/FlinkActionsE2eTest.java
 
b/flink-table-store-e2e-tests/src/test/java/org/apache/flink/table/store/tests/FlinkActionsE2eTest.java
index 5ed4e4a5..a5a0376f 100644
--- 
a/flink-table-store-e2e-tests/src/test/java/org/apache/flink/table/store/tests/FlinkActionsE2eTest.java
+++ 
b/flink-table-store-e2e-tests/src/test/java/org/apache/flink/table/store/tests/FlinkActionsE2eTest.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.store.tests;
 
 import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -257,7 +256,6 @@ public class FlinkActionsE2eTest extends E2eTestBase {
                 "2023-01-21, 1, 31");
     }
 
-    @Disabled
     @Test
     public void testMergeInto() throws Exception {
         String tableTDdl =
diff --git a/flink-table-store-flink/flink-table-store-flink-1.14/pom.xml 
b/flink-table-store-flink/flink-table-store-flink-1.14/pom.xml
index 8579b836..7a4d97be 100644
--- a/flink-table-store-flink/flink-table-store-flink-1.14/pom.xml
+++ b/flink-table-store-flink/flink-table-store-flink-1.14/pom.xml
@@ -79,6 +79,14 @@ under the License.
             <scope>test</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-hdfs-client</artifactId>
diff --git 
a/flink-table-store-flink/flink-table-store-flink-1.14/src/main/java/org/apache/flink/table/store/connector/utils/TableEnvironmentUtils.java
 
b/flink-table-store-flink/flink-table-store-flink-1.14/src/main/java/org/apache/flink/table/store/connector/utils/TableEnvironmentUtils.java
new file mode 100644
index 00000000..a4fd18bf
--- /dev/null
+++ 
b/flink-table-store-flink/flink-table-store-flink-1.14/src/main/java/org/apache/flink/table/store/connector/utils/TableEnvironmentUtils.java
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.table.store.connector.utils;
+
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+/** Utility methods for {@link TableEnvironment} and its subclasses. */
+public class TableEnvironmentUtils {
+
+    /**
+     * Invoke {@code 
TableEnvironmentImpl#executeInternal(List<Transformation<?>>, List<String>)}
+     * from a {@link StreamTableEnvironment} instance through reflecting.
+     */
+    public static void executeInternal(
+            StreamTableEnvironment tEnv,
+            List<Transformation<?>> transformations,
+            List<String> sinkIdentifierNames) {
+        Class<?> clazz = tEnv.getClass().getSuperclass();
+        try {
+            Method executeInternal =
+                    clazz.getDeclaredMethod("executeInternal", List.class, 
List.class);
+            executeInternal.setAccessible(true);
+
+            TableResult tableResult =
+                    (TableResult)
+                            executeInternal.invoke(tEnv, transformations, 
sinkIdentifierNames);
+            tableResult.await();
+        } catch (NoSuchMethodException e) {
+            throw new RuntimeException(
+                    "Failed to get 'TableEnvironmentImpl#executeInternal(List, 
List)' method "
+                            + "from given StreamTableEnvironment instance by 
Java reflection. This is unexpected.",
+                    e);
+        } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(
+                    "Failed to invoke 
'TableEnvironmentImpl#executeInternal(List, List)' method "
+                            + "from given StreamTableEnvironment instance by 
Java reflection. This is unexpected.",
+                    e);
+        } catch (ExecutionException | InterruptedException e) {
+            throw new RuntimeException("Failed to wait for insert job to 
finish.", e);
+        }
+    }
+}
diff --git 
a/flink-table-store-flink/flink-table-store-flink-1.14/src/test/java/org/apache/flink/table/store/connector/FlinkActionITCase.java
 
b/flink-table-store-flink/flink-table-store-flink-1.14/src/test/java/org/apache/flink/table/store/connector/FlinkActionITCase.java
new file mode 100644
index 00000000..08326546
--- /dev/null
+++ 
b/flink-table-store-flink/flink-table-store-flink-1.14/src/test/java/org/apache/flink/table/store/connector/FlinkActionITCase.java
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.table.store.connector;
+
+import org.apache.flink.table.store.connector.action.DeleteAction;
+import org.apache.flink.table.store.file.utils.BlockingIterator;
+import org.apache.flink.types.Row;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.table.planner.factories.TestValuesTableFactory.changelogRow;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase for Flink action. */
+public class FlinkActionITCase extends CatalogITCaseBase {
+
+    protected int defaultParallelism() {
+        return 1;
+    }
+
+    protected List<String> ddl() {
+        return Collections.singletonList("CREATE TABLE T (k INT, v STRING)");
+    }
+
+    @Test
+    public void testDeleteAction() throws Exception {
+        batchSql("INSERT INTO T VALUES (1, 'Hi'), (2, 'Hello'), (3, 'World')");
+
+        DeleteAction action = new DeleteAction(path, "default", "T", "k = 1");
+
+        BlockingIterator<Row, Row> iterator =
+                BlockingIterator.of(sEnv.executeSql("SELECT * FROM 
T").collect());
+
+        action.run();
+
+        assertThat(iterator.collect(4))
+                .containsExactlyInAnyOrder(
+                        changelogRow("+I", 1, "Hi"),
+                        changelogRow("+I", 2, "Hello"),
+                        changelogRow("+I", 3, "World"),
+                        changelogRow("-D", 1, "Hi"));
+        iterator.close();
+    }
+}
diff --git a/flink-table-store-flink/flink-table-store-flink-1.15/pom.xml 
b/flink-table-store-flink/flink-table-store-flink-1.15/pom.xml
index 6369c835..13917113 100644
--- a/flink-table-store-flink/flink-table-store-flink-1.15/pom.xml
+++ b/flink-table-store-flink/flink-table-store-flink-1.15/pom.xml
@@ -72,6 +72,14 @@ under the License.
             <scope>test</scope>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            
<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
             <artifactId>flink-connector-base</artifactId>
diff --git 
a/flink-table-store-flink/flink-table-store-flink-1.15/src/test/java/org/apache/flink/table/store/connector/FlinkActionITCase.java
 
b/flink-table-store-flink/flink-table-store-flink-1.15/src/test/java/org/apache/flink/table/store/connector/FlinkActionITCase.java
new file mode 100644
index 00000000..08326546
--- /dev/null
+++ 
b/flink-table-store-flink/flink-table-store-flink-1.15/src/test/java/org/apache/flink/table/store/connector/FlinkActionITCase.java
@@ -0,0 +1,63 @@
+/*
+ * 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.flink.table.store.connector;
+
+import org.apache.flink.table.store.connector.action.DeleteAction;
+import org.apache.flink.table.store.file.utils.BlockingIterator;
+import org.apache.flink.types.Row;
+
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.table.planner.factories.TestValuesTableFactory.changelogRow;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase for Flink action. */
+public class FlinkActionITCase extends CatalogITCaseBase {
+
+    protected int defaultParallelism() {
+        return 1;
+    }
+
+    protected List<String> ddl() {
+        return Collections.singletonList("CREATE TABLE T (k INT, v STRING)");
+    }
+
+    @Test
+    public void testDeleteAction() throws Exception {
+        batchSql("INSERT INTO T VALUES (1, 'Hi'), (2, 'Hello'), (3, 'World')");
+
+        DeleteAction action = new DeleteAction(path, "default", "T", "k = 1");
+
+        BlockingIterator<Row, Row> iterator =
+                BlockingIterator.of(sEnv.executeSql("SELECT * FROM 
T").collect());
+
+        action.run();
+
+        assertThat(iterator.collect(4))
+                .containsExactlyInAnyOrder(
+                        changelogRow("+I", 1, "Hi"),
+                        changelogRow("+I", 2, "Hello"),
+                        changelogRow("+I", 3, "World"),
+                        changelogRow("-D", 1, "Hi"));
+        iterator.close();
+    }
+}
diff --git 
a/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/ActionBase.java
 
b/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/ActionBase.java
index 2e9014c8..d0750c09 100644
--- 
a/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/ActionBase.java
+++ 
b/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/ActionBase.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.store.connector.action;
 
+import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
@@ -27,6 +28,7 @@ import org.apache.flink.table.store.catalog.CatalogContext;
 import org.apache.flink.table.store.connector.FlinkCatalog;
 import org.apache.flink.table.store.connector.LogicalTypeConversion;
 import org.apache.flink.table.store.connector.sink.FlinkSinkBuilder;
+import org.apache.flink.table.store.connector.utils.TableEnvironmentUtils;
 import org.apache.flink.table.store.file.catalog.Catalog;
 import org.apache.flink.table.store.file.catalog.CatalogFactory;
 import org.apache.flink.table.store.file.catalog.Identifier;
@@ -42,6 +44,7 @@ import org.apache.flink.table.types.logical.LogicalType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.stream.Collectors;
 
@@ -125,12 +128,20 @@ public abstract class ActionBase implements Action {
         return true;
     }
 
-    /** Sink {@link DataStream} dataStream to table. */
-    protected void sink(DataStream<RowData> dataStream) throws Exception {
-        new FlinkSinkBuilder((FileStoreTable) table)
-                .withInput(dataStream)
-                
.withLockFactory(Lock.factory(catalog.lockFactory().orElse(null), identifier))
-                .build();
-        env.execute();
+    /** Sink {@link DataStream} dataStream to table with Flink Table API in 
batch environment. */
+    protected void batchSink(DataStream<RowData> dataStream) {
+        List<Transformation<?>> transformations =
+                Collections.singletonList(
+                        new FlinkSinkBuilder((FileStoreTable) table)
+                                .withInput(dataStream)
+                                .withLockFactory(
+                                        Lock.factory(
+                                                
catalog.lockFactory().orElse(null), identifier))
+                                .build()
+                                .getTransformation());
+
+        List<String> sinkIdentifierNames = 
Collections.singletonList(identifier.getFullName());
+
+        TableEnvironmentUtils.executeInternal(tEnv, transformations, 
sinkIdentifierNames);
     }
 }
diff --git 
a/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/DeleteAction.java
 
b/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/DeleteAction.java
index 6d18d52d..5890a925 100644
--- 
a/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/DeleteAction.java
+++ 
b/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/DeleteAction.java
@@ -129,6 +129,6 @@ public class DeleteAction extends ActionBase {
                                     return rowData;
                                 });
 
-        sink(dataStream);
+        batchSink(dataStream);
     }
 }
diff --git 
a/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/MergeIntoAction.java
 
b/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/MergeIntoAction.java
index e131087d..93de9d26 100644
--- 
a/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/MergeIntoAction.java
+++ 
b/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/action/MergeIntoAction.java
@@ -454,7 +454,7 @@ public class MergeIntoAction extends ActionBase {
                         .collect(Collectors.toList());
 
         DataStream<RowData> firstDs = dataStreams.get(0);
-        
sink(firstDs.union(dataStreams.stream().skip(1).toArray(DataStream[]::new)));
+        
batchSink(firstDs.union(dataStreams.stream().skip(1).toArray(DataStream[]::new)));
     }
 
     private Optional<DataStream<RowData>> getMatchedUpsertDataStream() {
diff --git 
a/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/utils/TableEnvironmentUtils.java
 
b/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/utils/TableEnvironmentUtils.java
new file mode 100644
index 00000000..ce818330
--- /dev/null
+++ 
b/flink-table-store-flink/flink-table-store-flink-common/src/main/java/org/apache/flink/table/store/connector/utils/TableEnvironmentUtils.java
@@ -0,0 +1,66 @@
+/*
+ * 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.flink.table.store.connector.utils;
+
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+/** Utility methods for {@link TableEnvironment} and its subclasses. */
+public class TableEnvironmentUtils {
+
+    /**
+     * Invoke {@code 
TableEnvironmentImpl#executeInternal(List<Transformation<?>>, List<String>)}
+     * from a {@link StreamTableEnvironment} instance through reflecting.
+     */
+    public static void executeInternal(
+            StreamTableEnvironment tEnv,
+            List<Transformation<?>> transformations,
+            List<String> sinkIdentifierNames) {
+        Class<?> clazz = tEnv.getClass().getSuperclass().getSuperclass();
+        try {
+            Method executeInternal =
+                    clazz.getDeclaredMethod("executeInternal", List.class, 
List.class);
+            executeInternal.setAccessible(true);
+
+            TableResult tableResult =
+                    (TableResult)
+                            executeInternal.invoke(tEnv, transformations, 
sinkIdentifierNames);
+            tableResult.await();
+        } catch (NoSuchMethodException e) {
+            throw new RuntimeException(
+                    "Failed to get 'TableEnvironmentImpl#executeInternal(List, 
List)' method "
+                            + "from given StreamTableEnvironment instance by 
Java reflection. This is unexpected.",
+                    e);
+        } catch (IllegalAccessException | InvocationTargetException e) {
+            throw new RuntimeException(
+                    "Failed to invoke 
'TableEnvironmentImpl#executeInternal(List, List)' method "
+                            + "from given StreamTableEnvironment instance by 
Java reflection. This is unexpected.",
+                    e);
+        } catch (ExecutionException | InterruptedException e) {
+            throw new RuntimeException("Failed to wait for insert job to 
finish.", e);
+        }
+    }
+}

Reply via email to