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

wenjun pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/incubator-seatunnel.git


The following commit(s) were added to refs/heads/dev by this push:
     new bf62a679 [Improve][UT] add ut for Flink SQL module (#1836)
bf62a679 is described below

commit bf62a67965538f883885027e4aa6bc476d316243
Author: legendtkl <[email protected]>
AuthorDate: Tue May 10 14:35:06 2022 +0800

    [Improve][UT] add ut for Flink SQL module (#1836)
    
    * add ut for Flink SQL module
---
 .../apache/seatunnel/core/sql/job/Executor.java    | 22 +++++------
 .../seatunnel/core/sql/job/ExecutorTest.java       | 44 ++++++++++++++++++++++
 .../apache/seatunnel/core/sql/job/JobInfoTest.java | 34 +++++++++++++++++
 3 files changed, 89 insertions(+), 11 deletions(-)

diff --git 
a/seatunnel-core/seatunnel-core-flink-sql/src/main/java/org/apache/seatunnel/core/sql/job/Executor.java
 
b/seatunnel-core/seatunnel-core-flink-sql/src/main/java/org/apache/seatunnel/core/sql/job/Executor.java
index 3bd275d3..2fd9612f 100644
--- 
a/seatunnel-core/seatunnel-core-flink-sql/src/main/java/org/apache/seatunnel/core/sql/job/Executor.java
+++ 
b/seatunnel-core/seatunnel-core-flink-sql/src/main/java/org/apache/seatunnel/core/sql/job/Executor.java
@@ -20,6 +20,8 @@ package org.apache.seatunnel.core.sql.job;
 import org.apache.seatunnel.core.sql.splitter.SqlStatementSplitter;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
@@ -63,10 +65,10 @@ public class Executor {
 
         List<String> stmts = 
SqlStatementSplitter.normalizeStatements(workFlowContent);
         for (String stmt : stmts) {
-            Optional<String[]> optional = setOperationParse(stmt);
+            Optional<Pair<String, String>> optional = parseSetOperation(stmt);
             if (optional.isPresent()) {
-                String[] setOptionStrs = optional.get();
-                callSetOperation(configuration, setOptionStrs[0].trim(), 
setOptionStrs[1].trim());
+                Pair<String, String> setOptionPair = optional.get();
+                callSetOperation(configuration, setOptionPair.getLeft(), 
setOptionPair.getRight());
                 continue;
             }
             Operation op = stEnv.getParser().parse(stmt).get(0);
@@ -79,7 +81,8 @@ public class Executor {
         return statementSet;
     }
 
-    private static Optional<String[]> setOperationParse(String stmt) {
+    @VisibleForTesting
+    static Optional<Pair<String, String>> parseSetOperation(String stmt) {
         stmt = stmt.trim();
         Pattern pattern = Pattern.compile(FLINK_SQL_SET_MATCHING_REGEX, 
Pattern.CASE_INSENSITIVE | Pattern.DOTALL);
         final Matcher matcher = pattern.matcher(stmt);
@@ -93,15 +96,12 @@ public class Executor {
         return Optional.empty();
     }
 
-    private static Optional<String[]> operandConverter(String[] operands){
-        if (operands.length >= FLINK_SQL_SET_OPERANDS) {
-            if (operands[0] == null) {
-                return Optional.of(new String[0]);
-            }
-        } else {
+    private static Optional<Pair<String, String>> operandConverter(String[] 
operands){
+        if (operands.length != FLINK_SQL_SET_OPERANDS) {
             return Optional.empty();
         }
-        return Optional.of(new String[]{operands[1], operands[2]});
+
+        return Optional.of(Pair.of(operands[1].trim(), operands[2].trim()));
     }
 
     private static void callSetOperation(Configuration configuration, String 
key, String value) {
diff --git 
a/seatunnel-core/seatunnel-core-flink-sql/src/test/java/org/apache/seatunnel/core/sql/job/ExecutorTest.java
 
b/seatunnel-core/seatunnel-core-flink-sql/src/test/java/org/apache/seatunnel/core/sql/job/ExecutorTest.java
new file mode 100644
index 00000000..f5e2b496
--- /dev/null
+++ 
b/seatunnel-core/seatunnel-core-flink-sql/src/test/java/org/apache/seatunnel/core/sql/job/ExecutorTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.seatunnel.core.sql.job;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Optional;
+
+public class ExecutorTest {
+
+    @Test
+    public void testSetOperationParse() {
+        String stmt = "SET parallelism.default = 1";
+        Optional<Pair<String, String>> ops = Executor.parseSetOperation(stmt);
+        Assert.assertTrue(ops.isPresent());
+        Assert.assertEquals("parallelism.default", ops.get().getLeft());
+        Assert.assertEquals("1", ops.get().getRight());
+
+        stmt = "SET parallelism.default";
+        ops = Executor.parseSetOperation(stmt);
+        Assert.assertFalse(ops.isPresent());
+
+        stmt = "CREATE TABLE IF NOT EXISTS test (id INT, name VARCHAR)";
+        ops = Executor.parseSetOperation(stmt);
+        Assert.assertFalse(ops.isPresent());
+    }
+}
diff --git 
a/seatunnel-core/seatunnel-core-flink-sql/src/test/java/org/apache/seatunnel/core/sql/job/JobInfoTest.java
 
b/seatunnel-core/seatunnel-core-flink-sql/src/test/java/org/apache/seatunnel/core/sql/job/JobInfoTest.java
new file mode 100644
index 00000000..c87f499c
--- /dev/null
+++ 
b/seatunnel-core/seatunnel-core-flink-sql/src/test/java/org/apache/seatunnel/core/sql/job/JobInfoTest.java
@@ -0,0 +1,34 @@
+/*
+ * 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.seatunnel.core.sql.job;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class JobInfoTest {
+
+    @Test
+    public void testSubstitute() {
+        String sql = "SELECT * FROM TABLE WHERE NAME = '${name}' AND DATE = 
'${date}'";
+        JobInfo jobInfo = new JobInfo(sql);
+        jobInfo.substitute(Arrays.asList("name=seatunnel", "date=2019-01-01"));
+        Assert.assertEquals("SELECT * FROM TABLE WHERE NAME = 'seatunnel' AND 
DATE = '2019-01-01'", jobInfo.getJobContent());
+    }
+}

Reply via email to