You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@inlong.apache.org by do...@apache.org on 2022/08/26 05:03:22 UTC

[inlong] 01/08: [INLONG-5039][Manager] Support delete and update command, and query execution logs for manager client (#5519)

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

dockerzhang pushed a commit to branch release-1.3.0
in repository https://gitbox.apache.org/repos/asf/inlong.git

commit 7779926f81019ba27104e72f6632fbdbc0828d43
Author: Yizhou Yang <32...@users.noreply.github.com>
AuthorDate: Thu Aug 25 17:05:37 2022 +0800

    [INLONG-5039][Manager] Support delete and update command, and query execution logs for manager client (#5519)
---
 .../inlong/manager/client/cli/CommandToolMain.java |  5 +-
 .../inlong/manager/client/cli/CreateCommand.java   | 23 ++++--
 .../inlong/manager/client/cli/DeleteCommand.java   | 66 ++++++++++++++++
 .../inlong/manager/client/cli/LogCommand.java      | 87 ++++++++++++++++++++++
 .../cli/{CreateCommand.java => UpdateCommand.java} | 56 +++++++-------
 .../inlong/manager/client/cli/pojo/GroupInfo.java  |  2 +
 .../inlong/manager/client/cli/pojo/SinkInfo.java   |  2 +
 .../inlong/manager/client/cli/pojo/SourceInfo.java |  2 +
 .../inlong/manager/client/cli/pojo/StreamInfo.java |  2 +
 .../manager/client/cli/util/ClientUtils.java       |  5 +-
 .../inlong/manager/client/cli/TestCommand.java     | 25 ++++++-
 .../src/test/resources/create_group.json           | 29 +++-----
 .../src/test/resources/test_config.json            |  9 +++
 13 files changed, 249 insertions(+), 64 deletions(-)

diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CommandToolMain.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CommandToolMain.java
index 271016a4f..e8da1bb42 100644
--- a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CommandToolMain.java
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CommandToolMain.java
@@ -28,7 +28,6 @@ import java.util.Arrays;
 public class CommandToolMain {
 
     private final JCommander jcommander;
-
     @Parameter(names = {"-h", "--help"}, help = true, description = "Get all command about managerctl.")
     boolean help;
 
@@ -36,10 +35,12 @@ public class CommandToolMain {
         jcommander = new JCommander();
         jcommander.setProgramName("managerctl");
         jcommander.addObject(this);
-
         jcommander.addCommand("list", new ListCommand());
         jcommander.addCommand("describe", new DescribeCommand());
         jcommander.addCommand("create", new CreateCommand());
+        jcommander.addCommand("delete", new DeleteCommand());
+        jcommander.addCommand("update", new UpdateCommand());
+        jcommander.addCommand("log", new LogCommand());
     }
 
     public static void main(String[] args) {
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CreateCommand.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CreateCommand.java
index 0967c66d0..2a099f958 100644
--- a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CreateCommand.java
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CreateCommand.java
@@ -51,26 +51,37 @@ public class CreateCommand extends AbstractCommand {
 
         @Parameter(names = {"-f", "--file"},
                 converter = FileConverter.class,
-                required = true,
                 description = "json file")
         private File file;
 
+        @Parameter(names = {"-s"}, description = "optional log string to create file")
+        private String input;
+
         @Override
         void run() {
             try {
-                String fileContent = ClientUtils.readFile(file);
-                if (StringUtils.isBlank(fileContent)) {
-                    System.out.println("Create group failed: file was empty!");
-                    return;
+                String content;
+                if (input != null) {
+                    content = input;
+                } else {
+                    content = ClientUtils.readFile(file);
+                    if (StringUtils.isBlank(content)) {
+                        System.out.println("Create group failed: file was empty!");
+                        return;
+                    }
                 }
-                CreateGroupConf groupConf = objectMapper.readValue(fileContent, CreateGroupConf.class);
+                // first extract groupconfig from the file passed in
+                CreateGroupConf groupConf = objectMapper.readValue(content, CreateGroupConf.class);
+                // get the correspodning inlonggroup, a.k.a the task to execute
                 InlongClient inlongClient = ClientUtils.getClient();
                 InlongGroup group = inlongClient.forGroup(groupConf.getGroupInfo());
                 InlongStreamBuilder streamBuilder = group.createStream(groupConf.getStreamInfo());
+                // put in parameters:source and sink,stream fields, then initialize
                 streamBuilder.fields(groupConf.getStreamFieldList());
                 streamBuilder.source(groupConf.getStreamSource());
                 streamBuilder.sink(groupConf.getStreamSink());
                 streamBuilder.initOrUpdate();
+                // initialize the new stream group
                 group.init();
                 System.out.println("Create group success!");
             } catch (Exception e) {
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/DeleteCommand.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/DeleteCommand.java
new file mode 100644
index 000000000..e7746deb2
--- /dev/null
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/DeleteCommand.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.inlong.manager.client.cli;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import org.apache.inlong.manager.client.api.InlongClient;
+import org.apache.inlong.manager.client.api.InlongGroup;
+import org.apache.inlong.manager.client.cli.util.ClientUtils;
+
+import java.util.List;
+
+/**
+ * The delete command used for deleting inlong group instances.
+ * Please refer to the document for parameters
+ */
+@Parameters(commandDescription = "Delete resource by json file")
+public class DeleteCommand extends AbstractCommand {
+
+    @Parameter()
+    private List<String> params;
+
+    public DeleteCommand() {
+        super("delete");
+        jcommander.addCommand("group", new DeleteCommand.DeleteGroup());
+    }
+
+    @Parameters(commandDescription = "Delete group by group id")
+    private static class DeleteGroup extends AbstractCommandRunner {
+
+        @Parameter()
+        private List<String> params;
+
+        @Parameter(names = {"--group", "-g"}, required = true, description = "inlong group id")
+        private String inlongGroupId;
+
+        @Override
+        void run() {
+            // get the group and the corresponding context(snapshot)
+            // TODO: handle and/or classify the exceptions
+            try {
+                InlongClient inlongClient = ClientUtils.getClient();
+                InlongGroup group = inlongClient.getGroup(inlongGroupId);
+                group.delete();
+                System.out.println("delete group success");
+            } catch (Exception e) {
+                System.out.format("Delete group failed! message: %s \n", e.getMessage());
+            }
+        }
+    }
+}
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/LogCommand.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/LogCommand.java
new file mode 100644
index 000000000..dd6a8c15a
--- /dev/null
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/LogCommand.java
@@ -0,0 +1,87 @@
+/*
+ * 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.inlong.manager.client.cli;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.github.pagehelper.PageInfo;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.inlong.manager.client.api.inner.client.InlongGroupClient;
+import org.apache.inlong.manager.client.cli.pojo.GroupInfo;
+import org.apache.inlong.manager.client.cli.util.ClientUtils;
+import org.apache.inlong.manager.client.cli.util.PrintUtils;
+import org.apache.inlong.manager.pojo.group.InlongGroupBriefInfo;
+import org.apache.inlong.manager.pojo.group.InlongGroupPageRequest;
+
+import java.util.List;
+
+/**
+ * The log command was used to get log info for specified inlong groups.
+ */
+@Parameters(commandDescription = "Log resource")
+public class LogCommand extends AbstractCommand {
+
+    @Parameter()
+    private List<String> params;
+
+    public LogCommand() {
+        super("log");
+        jcommander.addCommand("group", new CreateGroup());
+    }
+
+    @Parameters(commandDescription = "Log group")
+    private static class CreateGroup extends AbstractCommandRunner {
+
+        @Parameter()
+        private List<String> params;
+
+        @Parameter(names = {"--query"}, required = true, description = "condition filters")
+        private String input;
+
+        @Override
+        void run() {
+            final int MAX_LOG_SIZE = 100;
+            try {
+                // for now only filter by one condition. TODO:support OR and AND, make a condition filter.
+                // sample input: inlongGroupId:test_group
+                if (StringUtils.isNotBlank(input)) {
+                    System.err.println("input cannot be empty, for example: inlongGroupId:test_group");
+                    return;
+                }
+                String[] inputs = input.split(":");
+                if (inputs.length < 2 || StringUtils.isBlank(inputs[1])) {
+                    System.err.println("the input must contain ':'");
+                    return;
+                }
+
+                ClientUtils.initClientFactory();
+                InlongGroupClient groupClient = ClientUtils.clientFactory.getGroupClient();
+                InlongGroupPageRequest pageRequest = new InlongGroupPageRequest();
+                pageRequest.setKeyword(inputs[1]);
+                PageInfo<InlongGroupBriefInfo> pageInfo = groupClient.listGroups(pageRequest);
+                if (pageInfo.getSize() > MAX_LOG_SIZE) {
+                    System.err.println("the log is too large to print, please change the filter condition");
+                    return;
+                }
+                PrintUtils.print(pageInfo.getList(), GroupInfo.class);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+}
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CreateCommand.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/UpdateCommand.java
similarity index 50%
copy from inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CreateCommand.java
copy to inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/UpdateCommand.java
index 0967c66d0..6e7ad8ec5 100644
--- a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/CreateCommand.java
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/UpdateCommand.java
@@ -19,63 +19,59 @@ package org.apache.inlong.manager.client.cli;
 
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.Parameters;
-import com.beust.jcommander.converters.FileConverter;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.inlong.manager.client.api.InlongClient;
 import org.apache.inlong.manager.client.api.InlongGroup;
-import org.apache.inlong.manager.client.api.InlongStreamBuilder;
-import org.apache.inlong.manager.client.cli.pojo.CreateGroupConf;
 import org.apache.inlong.manager.client.cli.util.ClientUtils;
+import org.apache.inlong.manager.pojo.sort.BaseSortConf;
 
 import java.io.File;
+import java.util.List;
 
 /**
- * Create resource by json file.
+ * The update command used to change the fields of inlong groups.
+ * Please refer to the document for parameters
  */
-@Parameters(commandDescription = "Create resource by json file")
-public class CreateCommand extends AbstractCommand {
+@Parameters(commandDescription = "Update resource by json file")
+public class UpdateCommand extends AbstractCommand {
 
     @Parameter()
-    private java.util.List<String> params;
+    private List<String> params;
 
-    public CreateCommand() {
-        super("create");
-        jcommander.addCommand("group", new CreateGroup());
+    public UpdateCommand() {
+        super("update");
+        jcommander.addCommand("group", new UpdateCommand.UpdateGroup());
     }
 
-    @Parameters(commandDescription = "Create group by json file")
-    private static class CreateGroup extends AbstractCommandRunner {
+    @Parameters(commandDescription = "Update group by json file")
+    private static class UpdateGroup extends AbstractCommandRunner {
 
         @Parameter()
-        private java.util.List<String> params;
+        private List<String> params;
 
-        @Parameter(names = {"-f", "--file"},
-                converter = FileConverter.class,
-                required = true,
-                description = "json file")
+        @Parameter(names = {"--group", "-g"}, required = true, description = "inlong group id")
+        private String inlongGroupId;
+
+        @Parameter(names = {"-c", "--config"},
+                required = true, description = "json file")
         private File file;
 
         @Override
         void run() {
             try {
+                InlongClient inlongClient = ClientUtils.getClient();
+                InlongGroup group = inlongClient.getGroup(inlongGroupId);
                 String fileContent = ClientUtils.readFile(file);
                 if (StringUtils.isBlank(fileContent)) {
-                    System.out.println("Create group failed: file was empty!");
+                    System.out.println("Update group failed: file was empty!");
                     return;
                 }
-                CreateGroupConf groupConf = objectMapper.readValue(fileContent, CreateGroupConf.class);
-                InlongClient inlongClient = ClientUtils.getClient();
-                InlongGroup group = inlongClient.forGroup(groupConf.getGroupInfo());
-                InlongStreamBuilder streamBuilder = group.createStream(groupConf.getStreamInfo());
-                streamBuilder.fields(groupConf.getStreamFieldList());
-                streamBuilder.source(groupConf.getStreamSource());
-                streamBuilder.sink(groupConf.getStreamSink());
-                streamBuilder.initOrUpdate();
-                group.init();
-                System.out.println("Create group success!");
+                // first extract groupconfig from the file passed in
+                BaseSortConf sortConf = objectMapper.readValue(fileContent, BaseSortConf.class);
+                group.update(sortConf);
+                System.out.println("update group success");
             } catch (Exception e) {
-                System.out.println("Create group failed!");
-                System.out.println(e.getMessage());
+                e.printStackTrace();
             }
         }
     }
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/GroupInfo.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/GroupInfo.java
index 27f4b4b0f..a85322955 100644
--- a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/GroupInfo.java
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/GroupInfo.java
@@ -17,6 +17,7 @@
 
 package org.apache.inlong.manager.client.cli.pojo;
 
+import com.fasterxml.jackson.annotation.JsonFormat;
 import lombok.Data;
 import org.apache.inlong.manager.client.cli.util.ParseStatus;
 
@@ -34,6 +35,7 @@ public class GroupInfo {
 
     @ParseStatus
     private String status;
+    @JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss")
     private Date modifyTime;
 
 }
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/SinkInfo.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/SinkInfo.java
index 2804be3f1..a714eefc2 100644
--- a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/SinkInfo.java
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/SinkInfo.java
@@ -17,6 +17,7 @@
 
 package org.apache.inlong.manager.client.cli.pojo;
 
+import com.fasterxml.jackson.annotation.JsonFormat;
 import lombok.Data;
 import org.apache.inlong.manager.client.cli.util.ParseStatus;
 
@@ -36,5 +37,6 @@ public class SinkInfo {
 
     @ParseStatus
     private String status;
+    @JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss")
     private Date modifyTime;
 }
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/SourceInfo.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/SourceInfo.java
index ebe62ae37..131bc95d3 100644
--- a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/SourceInfo.java
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/SourceInfo.java
@@ -17,6 +17,7 @@
 
 package org.apache.inlong.manager.client.cli.pojo;
 
+import com.fasterxml.jackson.annotation.JsonFormat;
 import lombok.Data;
 import org.apache.inlong.manager.client.cli.util.ParseStatus;
 
@@ -37,5 +38,6 @@ public class SourceInfo {
 
     @ParseStatus
     private String status;
+    @JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss")
     private Date modifyTime;
 }
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/StreamInfo.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/StreamInfo.java
index df48dcfab..4c9292d0b 100644
--- a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/StreamInfo.java
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/pojo/StreamInfo.java
@@ -17,6 +17,7 @@
 
 package org.apache.inlong.manager.client.cli.pojo;
 
+import com.fasterxml.jackson.annotation.JsonFormat;
 import lombok.Data;
 import org.apache.inlong.manager.client.cli.util.ParseStatus;
 
@@ -37,5 +38,6 @@ public class StreamInfo {
 
     @ParseStatus
     private String status;
+    @JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss")
     private Date modifyTime;
 }
diff --git a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/util/ClientUtils.java b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/util/ClientUtils.java
index 5ade5e425..133113b28 100644
--- a/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/util/ClientUtils.java
+++ b/inlong-manager/manager-client-tools/src/main/java/org/apache/inlong/manager/client/cli/util/ClientUtils.java
@@ -38,13 +38,10 @@ import java.util.Properties;
 public class ClientUtils {
 
     private static final String CONFIG_FILE = "application.properties";
-
+    public static ClientFactory clientFactory;
     private static ClientConfiguration configuration;
-
     private static String serviceUrl;
 
-    public static ClientFactory clientFactory;
-
     /**
      * Get an inlong client instance.
      */
diff --git a/inlong-manager/manager-client-tools/src/test/java/org/apache/inlong/manager/client/cli/TestCommand.java b/inlong-manager/manager-client-tools/src/test/java/org/apache/inlong/manager/client/cli/TestCommand.java
index 79e198aba..3f6f3cbe0 100644
--- a/inlong-manager/manager-client-tools/src/test/java/org/apache/inlong/manager/client/cli/TestCommand.java
+++ b/inlong-manager/manager-client-tools/src/test/java/org/apache/inlong/manager/client/cli/TestCommand.java
@@ -34,21 +34,40 @@ public class TestCommand {
         log.info("client tools cannot run the unit tests, as the application.properties not exist");
     }
 
-    // @Test
+    @Test
     public void testListGroup() {
         String[] arg = {"list", "group"};
         Assertions.assertTrue(inlongAdminTool.run(arg));
     }
 
-    // @Test
+    @Test
     public void testDescribeGroup() {
         String[] arg = {"describe", "group", "-g", "test", "-s", "130"};
         Assertions.assertTrue(inlongAdminTool.run(arg));
     }
 
-    // @Test
+    @Test
     public void testCreateGroup() {
         String[] arg = {"create", "group", "-f", "src/test/resources/create_group.json"};
         Assertions.assertTrue(inlongAdminTool.run(arg));
     }
+
+    @Test
+    public void testDeleteGroup() throws Exception {
+        String[] arg = {"delete", "group", "-g", "test_group"};
+        Assertions.assertTrue(inlongAdminTool.run(arg));
+    }
+
+    @Test
+    public void testUpdateGroup() throws Exception {
+        String[] arg = {"update", "group", "-g", "test_group", "-c", "src/test/resources/test_config.json"};
+        Assertions.assertTrue(inlongAdminTool.run(arg));
+    }
+
+    @Test
+    public void testLogGroup() throws Exception {
+        String[] arg = {"log", "group", "--query", "inlongGroupId:test_group"};
+        Assertions.assertTrue(inlongAdminTool.run(arg));
+    }
+
 }
diff --git a/inlong-manager/manager-client-tools/src/test/resources/create_group.json b/inlong-manager/manager-client-tools/src/test/resources/create_group.json
index e91ed573c..c9f2b0395 100644
--- a/inlong-manager/manager-client-tools/src/test/resources/create_group.json
+++ b/inlong-manager/manager-client-tools/src/test/resources/create_group.json
@@ -13,11 +13,7 @@
     "retentionTimeUnit": "hours",
     "retentionSize": -1,
     "retentionSizeUnit": "MB",
-    "inlongClusterTag": "default_cluster",
-    "sortConf": {
-      "sortType": "flink",
-      "serviceUrl": "127.0.0.1"
-    }
+    "inlongClusterTag": "default_cluster"
   },
   "streamInfo": {
     "dataType": "CSV",
@@ -41,21 +37,16 @@
   "streamSource": {
     "inlongStreamId": "test_stream",
     "sourceName": "source_name",
-    "hostname": "127.0.0.1",
-    "port": 3306,
-    "user": "root",
-    "password": "123456",
-    "historyFilename": "/data/inlong-agent/.history",
-    "serverTimezone": "UTC",
-    "intervalMs": 1000,
-    "allMigration": false,
-    "databaseWhiteList": "etl",
-    "tableWhiteList": "person",
     "inlongGroupId": "test_group",
-    "sourceType": "BINLOG",
-    "specificOffsetFile": "mysql-bin.000001",
-    "specificOffsetPos": 6972,
-    "snapshotMode": "schema_only_recovery"
+    "sourceType": "FILE",
+    "agentIp": "127.0.0.1",
+    "uuid": "127.0.0.1",
+    "clusterId": "123",
+    "inlongClusterName": "hi",
+    "dataNodeName": "hi2",
+    "serializationType": "csv",
+    "snapshot": "hi3",
+    "properties": {}
   },
   "streamSink": {
     "inlongStreamId": "test_stream",
diff --git a/inlong-manager/manager-client-tools/src/test/resources/test_config.json b/inlong-manager/manager-client-tools/src/test/resources/test_config.json
new file mode 100644
index 000000000..56fc72a19
--- /dev/null
+++ b/inlong-manager/manager-client-tools/src/test/resources/test_config.json
@@ -0,0 +1,9 @@
+{
+  "FlinkSortConf": {
+    "sortType": "flink",
+    "authentication": "NONE",
+    "serviceUrl": "127.0.0.1:8123",
+    "region": "beijing",
+    "properties": {}
+  }
+}