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/12 02:33:56 UTC

[inlong] branch master updated: [INLONG-5493][Manager] Add user API in the manager client (#5494)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 2f3639c1b [INLONG-5493][Manager] Add user API in the manager client (#5494)
2f3639c1b is described below

commit 2f3639c1be3387cd0a0b577c67779bb5642bf611
Author: haifxu <xh...@gmail.com>
AuthorDate: Fri Aug 12 10:33:52 2022 +0800

    [INLONG-5493][Manager] Add user API in the manager client (#5494)
---
 .../org/apache/inlong/manager/client/api/User.java |  72 +++++++++++++
 .../inlong/manager/client/api/impl/UserImpl.java   |  75 +++++++++++++
 .../client/api/inner/client/ClientFactory.java     |   3 +
 .../client/api/inner/client/UserClient.java        | 117 +++++++++++++++++++++
 .../inlong/manager/client/api/service/UserApi.java |  51 +++++++++
 .../client/api/inner/ClientFactoryTest.java        |  77 ++++++++++++++
 .../apache/inlong/manager/pojo/user/UserInfo.java  |   6 ++
 7 files changed, 401 insertions(+)

diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/User.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/User.java
new file mode 100644
index 000000000..21c091651
--- /dev/null
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/User.java
@@ -0,0 +1,72 @@
+/*
+ * 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.api;
+
+import com.github.pagehelper.PageInfo;
+import org.apache.inlong.manager.pojo.user.UserInfo;
+import org.apache.inlong.manager.pojo.user.UserRequest;
+
+public interface User {
+
+    /**
+     * get current user
+     *
+     * @return user info
+     */
+    UserInfo currentUser();
+
+    /**
+     * Save user info
+     *
+     * @param userInfo user info request
+     * @return user id after saving
+     */
+    Integer register(UserRequest userInfo);
+
+    /**
+     * Get user info by user id
+     *
+     * @param id user id
+     * @return user info
+     */
+    UserInfo getById(Integer id);
+
+    /**
+     * List all users basic info by request condition
+     *
+     * @param request request
+     * @return user info list
+     */
+    PageInfo<UserInfo> list(UserRequest request);
+
+    /**
+     * Update user info
+     *
+     * @param userInfo user info request
+     * @return user id
+     */
+    Integer update(UserRequest userInfo);
+
+    /**
+     * Delete user by id
+     *
+     * @param id user id
+     * @return whether succeed
+     */
+    Boolean delete(Integer id);
+}
diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/impl/UserImpl.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/impl/UserImpl.java
new file mode 100644
index 000000000..712854324
--- /dev/null
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/impl/UserImpl.java
@@ -0,0 +1,75 @@
+/*
+ * 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.api.impl;
+
+import com.github.pagehelper.PageInfo;
+import org.apache.inlong.manager.client.api.ClientConfiguration;
+import org.apache.inlong.manager.client.api.User;
+import org.apache.inlong.manager.client.api.inner.client.ClientFactory;
+import org.apache.inlong.manager.client.api.inner.client.UserClient;
+import org.apache.inlong.manager.client.api.util.ClientUtils;
+import org.apache.inlong.manager.common.util.Preconditions;
+import org.apache.inlong.manager.pojo.user.UserInfo;
+import org.apache.inlong.manager.pojo.user.UserRequest;
+
+public class UserImpl implements User {
+
+    private final UserClient userClient;
+
+    public UserImpl(ClientConfiguration configuration) {
+        ClientFactory clientFactory = ClientUtils.getClientFactory(configuration);
+        this.userClient = clientFactory.getUserClient();
+    }
+
+    @Override
+    public UserInfo currentUser() {
+        return userClient.currentUser();
+    }
+
+    @Override
+    public Integer register(UserRequest userInfo) {
+        Preconditions.checkNotEmpty(userInfo.getName(), "username cannot be empty");
+        Preconditions.checkNotEmpty(userInfo.getPassword(), "password cannot be empty");
+        return userClient.register(userInfo);
+    }
+
+    @Override
+    public UserInfo getById(Integer id) {
+        Preconditions.checkNotNull(id, "user id cannot be null");
+        return userClient.getById(id);
+    }
+
+    @Override
+    public PageInfo<UserInfo> list(UserRequest request) {
+        Preconditions.checkNotNull(request, "request cannot be null");
+        return userClient.list(request);
+    }
+
+    @Override
+    public Integer update(UserRequest userInfo) {
+        Preconditions.checkNotNull(userInfo, "userinfo cannot be null");
+        Preconditions.checkNotNull(userInfo.getId(), "user id cannot be null");
+        return userClient.update(userInfo);
+    }
+
+    @Override
+    public Boolean delete(Integer id) {
+        Preconditions.checkNotNull(id, "user id cannot be null");
+        return userClient.delete(id);
+    }
+}
diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/ClientFactory.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/ClientFactory.java
index 96ca880c7..bd8589cb1 100644
--- a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/ClientFactory.java
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/ClientFactory.java
@@ -42,6 +42,8 @@ public class ClientFactory {
 
     private final DataNodeClient dataNodeClient;
 
+    private final UserClient userClient;
+
     public ClientFactory(ClientConfiguration configuration) {
         groupClient = new InlongGroupClient(configuration);
         streamClient = new InlongStreamClient(configuration);
@@ -51,5 +53,6 @@ public class ClientFactory {
         transformClient = new StreamTransformClient(configuration);
         workflowClient = new WorkflowClient(configuration);
         dataNodeClient = new DataNodeClient(configuration);
+        userClient = new UserClient(configuration);
     }
 }
diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/UserClient.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/UserClient.java
new file mode 100644
index 000000000..c7c666b0c
--- /dev/null
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/inner/client/UserClient.java
@@ -0,0 +1,117 @@
+/*
+ * 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.api.inner.client;
+
+import com.github.pagehelper.PageInfo;
+import org.apache.inlong.manager.client.api.ClientConfiguration;
+import org.apache.inlong.manager.client.api.service.UserApi;
+import org.apache.inlong.manager.client.api.util.ClientUtils;
+import org.apache.inlong.manager.common.util.Preconditions;
+import org.apache.inlong.manager.pojo.common.Response;
+import org.apache.inlong.manager.pojo.user.UserInfo;
+import org.apache.inlong.manager.pojo.user.UserRequest;
+
+/**
+ * Client for {@link UserApi}.
+ */
+public class UserClient {
+
+    private final UserApi userApi;
+
+    public UserClient(ClientConfiguration configuration) {
+        userApi = ClientUtils.createRetrofit(configuration).create(UserApi.class);
+    }
+
+    /**
+     * get current user
+     *
+     * @return user info
+     */
+    public UserInfo currentUser() {
+        Response<UserInfo> response = ClientUtils.executeHttpCall(userApi.currentUser());
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Save user info
+     *
+     * @param userInfo user info request
+     * @return user id after saving
+     */
+    public Integer register(UserRequest userInfo) {
+        Preconditions.checkNotEmpty(userInfo.getName(), "username cannot be empty");
+        Preconditions.checkNotEmpty(userInfo.getPassword(), "password cannot be empty");
+        Response<Integer> response = ClientUtils.executeHttpCall(userApi.register(userInfo));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Get user info by user id
+     *
+     * @param id user id
+     * @return user info
+     */
+    public UserInfo getById(Integer id) {
+        Preconditions.checkNotNull(id, "user id cannot be null");
+        Response<UserInfo> response = ClientUtils.executeHttpCall(userApi.getById(id));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * List all users basic info by request condition
+     *
+     * @param request request
+     * @return user info list
+     */
+    public PageInfo<UserInfo> list(UserRequest request) {
+        Preconditions.checkNotNull(request, "request cannot be null");
+        Response<PageInfo<UserInfo>> response = ClientUtils.executeHttpCall(userApi.list(request));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Update user info
+     *
+     * @param userInfo user info request
+     * @return user id
+     */
+    public Integer update(UserRequest userInfo) {
+        Preconditions.checkNotNull(userInfo, "userinfo cannot be null");
+        Preconditions.checkNotNull(userInfo.getId(), "user id cannot be null");
+        Response<Integer> response = ClientUtils.executeHttpCall(userApi.update(userInfo));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+
+    /**
+     * Delete user by id
+     *
+     * @param id user id
+     * @return whether succeed
+     */
+    public Boolean delete(Integer id) {
+        Preconditions.checkNotNull(id, "user id cannot be null");
+        Response<Boolean> response = ClientUtils.executeHttpCall(userApi.delete(id));
+        ClientUtils.assertRespSuccess(response);
+        return response.getData();
+    }
+}
diff --git a/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/UserApi.java b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/UserApi.java
new file mode 100644
index 000000000..cd6708769
--- /dev/null
+++ b/inlong-manager/manager-client/src/main/java/org/apache/inlong/manager/client/api/service/UserApi.java
@@ -0,0 +1,51 @@
+/*
+ * 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.api.service;
+
+import com.github.pagehelper.PageInfo;
+import org.apache.inlong.manager.pojo.common.Response;
+import org.apache.inlong.manager.pojo.user.UserInfo;
+import org.apache.inlong.manager.pojo.user.UserRequest;
+import retrofit2.Call;
+import retrofit2.http.Body;
+import retrofit2.http.DELETE;
+import retrofit2.http.GET;
+import retrofit2.http.POST;
+import retrofit2.http.Path;
+import retrofit2.http.Query;
+
+public interface UserApi {
+
+    @POST("user/currentUser")
+    Call<Response<UserInfo>> currentUser();
+
+    @POST("user/register")
+    Call<Response<Integer>> register(@Body UserRequest userInfo);
+
+    @GET("user/get/{id}")
+    Call<Response<UserInfo>> getById(@Path("id") Integer id);
+
+    @GET("user/listAll")
+    Call<Response<PageInfo<UserInfo>>> list(UserRequest request);
+
+    @POST("user/update")
+    Call<Response<Integer>> update(@Body UserRequest userInfo);
+
+    @DELETE("user/delete")
+    Call<Response<Boolean>> delete(@Query("id") Integer id);
+}
diff --git a/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/ClientFactoryTest.java b/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/ClientFactoryTest.java
index 59e200338..203fd10ae 100644
--- a/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/ClientFactoryTest.java
+++ b/inlong-manager/manager-client/src/test/java/org/apache/inlong/manager/client/api/inner/ClientFactoryTest.java
@@ -33,6 +33,7 @@ import org.apache.inlong.manager.client.api.inner.client.InlongGroupClient;
 import org.apache.inlong.manager.client.api.inner.client.InlongStreamClient;
 import org.apache.inlong.manager.client.api.inner.client.StreamSinkClient;
 import org.apache.inlong.manager.client.api.inner.client.StreamSourceClient;
+import org.apache.inlong.manager.client.api.inner.client.UserClient;
 import org.apache.inlong.manager.client.api.util.ClientUtils;
 import org.apache.inlong.manager.common.auth.DefaultAuthentication;
 import org.apache.inlong.manager.common.auth.TokenAuthentication;
@@ -41,6 +42,7 @@ import org.apache.inlong.manager.common.consts.MQType;
 import org.apache.inlong.manager.common.consts.SinkType;
 import org.apache.inlong.manager.common.consts.SourceType;
 import org.apache.inlong.manager.common.enums.ClusterType;
+import org.apache.inlong.manager.common.enums.UserTypeEnum;
 import org.apache.inlong.manager.common.util.JsonUtils;
 import org.apache.inlong.manager.pojo.cluster.BindTagRequest;
 import org.apache.inlong.manager.pojo.cluster.ClusterInfo;
@@ -81,6 +83,8 @@ import org.apache.inlong.manager.pojo.stream.InlongStreamBriefInfo;
 import org.apache.inlong.manager.pojo.stream.InlongStreamInfo;
 import org.apache.inlong.manager.pojo.stream.InlongStreamResponse;
 import org.apache.inlong.manager.pojo.stream.StreamField;
+import org.apache.inlong.manager.pojo.user.UserInfo;
+import org.apache.inlong.manager.pojo.user.UserRequest;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeAll;
@@ -113,6 +117,7 @@ class ClientFactoryTest {
     private static StreamSinkClient sinkClient;
     private static InlongClusterClient clusterClient;
     private static DataNodeClient dataNodeClient;
+    private static UserClient userClient;
 
     @BeforeAll
     static void setup() {
@@ -132,6 +137,7 @@ class ClientFactoryTest {
         streamClient = clientFactory.getStreamClient();
         clusterClient = clientFactory.getClusterClient();
         dataNodeClient = clientFactory.getDataNodeClient();
+        userClient = clientFactory.getUserClient();
     }
 
     @AfterAll
@@ -1040,4 +1046,75 @@ class ClientFactoryTest {
         Boolean isUpdate = dataNodeClient.delete(1);
         Assertions.assertTrue(isUpdate);
     }
+
+    @Test
+    void testRegisterUser() {
+        stubFor(
+                post(urlMatching("/inlong/manager/api/user/register.*"))
+                        .willReturn(
+                                okJson(JsonUtils.toJsonString(
+                                        Response.success(1))
+                                )
+                        )
+        );
+        UserRequest request = new UserRequest();
+        request.setName("test_user");
+        request.setPassword("test_pwd");
+        Integer userId = userClient.register(request);
+        Assertions.assertEquals(1, userId);
+    }
+
+    @Test
+    void testGetUserById() {
+        UserInfo userInfo = UserInfo.builder()
+                .id(1)
+                .name("test_user")
+                .password("test_pwd")
+                .build();
+
+        stubFor(
+                get(urlMatching("/inlong/manager/api/user/get/1.*"))
+                        .willReturn(
+                                okJson(JsonUtils.toJsonString(
+                                        Response.success(userInfo))
+                                )
+                        )
+        );
+        UserInfo info = userClient.getById(1);
+        Assertions.assertEquals(info.getId(), 1);
+    }
+
+    @Test
+    void testUpdateUser() {
+        stubFor(
+                post(urlMatching("/inlong/manager/api/user/update.*"))
+                        .willReturn(
+                                okJson(JsonUtils.toJsonString(
+                                        Response.success(1))
+                                )
+                        )
+        );
+        UserRequest request = new UserRequest();
+        request.setId(1);
+        request.setName("test_user");
+        request.setPassword("test_pwd");
+        request.setNewPassword("test_new_pwd");
+        request.setAccountType(UserTypeEnum.ADMIN.getCode());
+        Integer userId = userClient.update(request);
+        Assertions.assertEquals(userId, 1);
+    }
+
+    @Test
+    void testDeleteUser() {
+        stubFor(
+                delete(urlMatching("/inlong/manager/api/user/delete.*"))
+                        .willReturn(
+                                okJson(JsonUtils.toJsonString(
+                                        Response.success(true))
+                                )
+                        )
+        );
+        Boolean isDelete = userClient.delete(1);
+        Assertions.assertTrue(isDelete);
+    }
 }
diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/user/UserInfo.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/user/UserInfo.java
index acb69098b..ee8dd6f12 100644
--- a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/user/UserInfo.java
+++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/user/UserInfo.java
@@ -20,7 +20,10 @@ package org.apache.inlong.manager.pojo.user;
 import com.fasterxml.jackson.annotation.JsonFormat;
 import io.swagger.annotations.ApiModel;
 import io.swagger.annotations.ApiModelProperty;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
 import lombok.Data;
+import lombok.NoArgsConstructor;
 
 import javax.validation.constraints.Min;
 import javax.validation.constraints.NotNull;
@@ -31,6 +34,9 @@ import java.util.Set;
  * User info
  */
 @Data
+@Builder
+@AllArgsConstructor
+@NoArgsConstructor
 @ApiModel("User info")
 public class UserInfo {