fsk119 commented on code in PR #21525:
URL: https://github.com/apache/flink/pull/21525#discussion_r1057036923


##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/util/SqlGatewayRestAPIVersion.java:
##########
@@ -34,8 +39,14 @@
         implements RestAPIVersion<SqlGatewayRestAPIVersion>, EndpointVersion {
     // The bigger the ordinal(its position in enum declaration), the higher 
the level of the
     // version.
+
+    // V0 is just for test
     V0(false, false),
+

Review Comment:
   // V1 introduces basic API for rest endpoint



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpointITCase.java:
##########
@@ -58,154 +58,139 @@
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import java.util.stream.Collectors;
 
 import static 
org.apache.flink.table.gateway.rest.util.RestConfigUtils.getBaseConfig;
 import static 
org.apache.flink.table.gateway.rest.util.RestConfigUtils.getFlinkConfig;
+import static 
org.apache.flink.table.gateway.rest.util.SqlGatewayRestClientAndEndpointUtils.TestRestClient.getTestRestClient;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** IT cases for {@link SqlGatewayRestEndpoint}. */
 class SqlGatewayRestEndpointITCase {
 
-    private static final SqlGatewayService service = null;
-
-    private static RestServerEndpoint serverEndpoint;
-    private static RestClient restClient;
+    private static SqlGatewayRestEndpoint serverEndpoint;
+    private static TestRestClient restClient;
     private static InetSocketAddress serverAddress;
 
-    private static TestBadCaseHandler testHandler;
-    private static TestVersionSelectionHeaders1 header1;
-    private static TestVersionSelectionHeaders2 header2;
     private static TestBadCaseHeaders badCaseHeader;
-    private static TestVersionHandler testVersionHandler1;
-    private static TestVersionHandler testVersionHandler2;
+    private static TestBadCaseHandler testHandler;
+
+    private static TestVersionSelectionHeaders0 header0;
+    private static TestVersionSelectionHeaders12 header12;
+
+    private static TestVersionHandler testVersionHandler0;
+    private static TestVersionHandler testVersionHandler12;
 
     private static Configuration config;
     private static final Time timeout = Time.seconds(10L);
 
     @BeforeEach
     void setup() throws Exception {
         // Test version cases
-        header1 = new TestVersionSelectionHeaders1();
-        header2 = new TestVersionSelectionHeaders2();
-        testVersionHandler1 = new TestVersionHandler(service, header1);
-        testVersionHandler2 = new TestVersionHandler(service, header2);
+        header0 = new TestVersionSelectionHeaders0();
+        header12 = new TestVersionSelectionHeaders12();
+        testVersionHandler0 = new TestVersionHandler(header0);
+        testVersionHandler12 = new TestVersionHandler(header12);
 
         // Test exception cases
         badCaseHeader = new TestBadCaseHeaders();
-        testHandler = new TestBadCaseHandler(service);
+        testHandler = new TestBadCaseHandler();
 
         // Init
         final String address = 
InetAddress.getLoopbackAddress().getHostAddress();
         config = getBaseConfig(getFlinkConfig(address, address, "0"));
         serverEndpoint =
-                TestingSqlGatewayRestEndpoint.builder(config, service)
+                TestSqlGatewayRestEndpoint.builder(config)
                         .withHandler(badCaseHeader, testHandler)
-                        .withHandler(header1, testVersionHandler1)
-                        .withHandler(header2, testVersionHandler2)
+                        .withHandler(header0, testVersionHandler0)
+                        .withHandler(header12, testVersionHandler12)
                         .buildAndStart();
 
-        restClient =
-                new RestClient(
-                        config,
-                        Executors.newFixedThreadPool(
-                                1, new 
ExecutorThreadFactory("rest-client-thread-pool")));
+        restClient = getTestRestClient();
         serverAddress = serverEndpoint.getServerAddress();
     }
 
     @AfterEach
     void stop() throws Exception {
-
         if (restClient != null) {
-            restClient.shutdown(timeout);
+            restClient.shutdown();
             restClient = null;
         }
 
         if (serverEndpoint != null) {
-            serverEndpoint.closeAsync().get(timeout.getSize(), 
timeout.getUnit());
+            serverEndpoint.stop();
             serverEndpoint = null;
         }
     }
 
     /** Test that {@link SqlGatewayMessageHeaders} can identify the version 
correctly. */
     @Test
     void testSqlGatewayMessageHeaders() throws Exception {
-        // The header only support V1, but send request by V0
+        // The header can't support V0, but sends request by V0
         assertThatThrownBy(
                         () ->
                                 restClient.sendRequest(
                                         serverAddress.getHostName(),
                                         serverAddress.getPort(),
-                                        header2,
+                                        header12,
                                         EmptyMessageParameters.getInstance(),
                                         EmptyRequestBody.getInstance(),
                                         Collections.emptyList(),
                                         SqlGatewayRestAPIVersion.V0))
-                .isInstanceOf(IllegalArgumentException.class);
-
-        // The header only support V1, send request by V1
+                .satisfies(
+                        FlinkAssertions.anyCauseMatches(
+                                IllegalArgumentException.class,
+                                String.format(
+                                        "The requested version V0 is not 
supported by the request (method=%s URL=%s). Supported versions are: %s.",
+                                        header12.getHttpMethod(),
+                                        header12.getTargetRestEndpointURL(),
+                                        
header12.getSupportedAPIVersions().stream()
+                                                
.map(RestAPIVersion::getURLVersionPrefix)
+                                                
.collect(Collectors.joining(",")))));
+
+        // The header only supports V0, sends request by V0
         CompletableFuture<TestResponse> specifiedVersionResponse =
                 restClient.sendRequest(
                         serverAddress.getHostName(),
                         serverAddress.getPort(),
-                        header2,
+                        header0,
                         EmptyMessageParameters.getInstance(),
                         EmptyRequestBody.getInstance(),
                         Collections.emptyList(),
-                        SqlGatewayRestAPIVersion.V1);
+                        SqlGatewayRestAPIVersion.V0);
 
-        TestResponse testResponse1 = specifiedVersionResponse.get(5, 
TimeUnit.SECONDS);
-        assertThat(testResponse1.getStatus()).isEqualTo("V1");
+        TestResponse testResponse1 =
+                specifiedVersionResponse.get(timeout.getSize(), 
timeout.getUnit());
+        assertThat(testResponse1.getStatus()).isEqualTo("V0");
 
-        // The header only support V1, send request by latest version V1
+        // The header supports V1 and V2, lets the client get the latest 
version as default
         CompletableFuture<TestResponse> unspecifiedVersionResponse =
                 restClient.sendRequest(
                         serverAddress.getHostName(),
                         serverAddress.getPort(),
-                        header2,
+                        header12,
                         EmptyMessageParameters.getInstance(),
                         EmptyRequestBody.getInstance(),
                         Collections.emptyList());
 
-        TestResponse testResponse2 = unspecifiedVersionResponse.get(5, 
TimeUnit.SECONDS);
-        assertThat(testResponse2.getStatus()).isEqualTo("V1");
+        TestResponse testResponse2 =
+                unspecifiedVersionResponse.get(timeout.getSize(), 
timeout.getUnit());
+        assertThat(testResponse2.getStatus()).isEqualTo("V2");

Review Comment:
   This part is same to `testDefaultVersionRouting`? If so, I think we can 
remove here.
   
   nit: BTW, it's better to introduce a case that API is only available in the 
high version, which is the actual case. With the default version routing, the 
request works.



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/RestAPIITCaseBase.java:
##########
@@ -20,15 +20,13 @@
 
 import org.apache.flink.api.common.time.Time;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.rest.RestClient;
 import org.apache.flink.runtime.rest.messages.MessageHeaders;
 import org.apache.flink.runtime.rest.messages.MessageParameters;
 import org.apache.flink.runtime.rest.messages.RequestBody;
 import org.apache.flink.runtime.rest.messages.ResponseBody;
+import 
org.apache.flink.table.gateway.rest.util.SqlGatewayRestClientAndEndpointUtils.TestRestClient;
 import org.apache.flink.table.gateway.service.utils.SqlGatewayServiceExtension;
 import org.apache.flink.test.junit5.MiniClusterExtension;
-import org.apache.flink.util.ExecutorUtils;
-import org.apache.flink.util.concurrent.ExecutorThreadFactory;
 
 import org.jetbrains.annotations.Nullable;

Review Comment:
   Align code style with others. Use `import javax.annotation.Nullable;` 
instead. Please also help to fix the same problems in the 
`SqlGatewayRestEndpointITCase`.



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpointITCase.java:
##########
@@ -58,154 +58,139 @@
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
+import java.util.stream.Collectors;
 
 import static 
org.apache.flink.table.gateway.rest.util.RestConfigUtils.getBaseConfig;
 import static 
org.apache.flink.table.gateway.rest.util.RestConfigUtils.getFlinkConfig;
+import static 
org.apache.flink.table.gateway.rest.util.SqlGatewayRestClientAndEndpointUtils.TestRestClient.getTestRestClient;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** IT cases for {@link SqlGatewayRestEndpoint}. */
 class SqlGatewayRestEndpointITCase {
 
-    private static final SqlGatewayService service = null;
-
-    private static RestServerEndpoint serverEndpoint;
-    private static RestClient restClient;
+    private static SqlGatewayRestEndpoint serverEndpoint;
+    private static TestRestClient restClient;
     private static InetSocketAddress serverAddress;
 
-    private static TestBadCaseHandler testHandler;
-    private static TestVersionSelectionHeaders1 header1;
-    private static TestVersionSelectionHeaders2 header2;
     private static TestBadCaseHeaders badCaseHeader;
-    private static TestVersionHandler testVersionHandler1;
-    private static TestVersionHandler testVersionHandler2;
+    private static TestBadCaseHandler testHandler;
+
+    private static TestVersionSelectionHeaders0 header0;
+    private static TestVersionSelectionHeaders12 header12;
+
+    private static TestVersionHandler testVersionHandler0;
+    private static TestVersionHandler testVersionHandler12;
 
     private static Configuration config;
     private static final Time timeout = Time.seconds(10L);
 
     @BeforeEach
     void setup() throws Exception {
         // Test version cases
-        header1 = new TestVersionSelectionHeaders1();
-        header2 = new TestVersionSelectionHeaders2();
-        testVersionHandler1 = new TestVersionHandler(service, header1);
-        testVersionHandler2 = new TestVersionHandler(service, header2);
+        header0 = new TestVersionSelectionHeaders0();
+        header12 = new TestVersionSelectionHeaders12();
+        testVersionHandler0 = new TestVersionHandler(header0);
+        testVersionHandler12 = new TestVersionHandler(header12);
 
         // Test exception cases
         badCaseHeader = new TestBadCaseHeaders();
-        testHandler = new TestBadCaseHandler(service);
+        testHandler = new TestBadCaseHandler();
 
         // Init
         final String address = 
InetAddress.getLoopbackAddress().getHostAddress();
         config = getBaseConfig(getFlinkConfig(address, address, "0"));
         serverEndpoint =
-                TestingSqlGatewayRestEndpoint.builder(config, service)
+                TestSqlGatewayRestEndpoint.builder(config)
                         .withHandler(badCaseHeader, testHandler)
-                        .withHandler(header1, testVersionHandler1)
-                        .withHandler(header2, testVersionHandler2)
+                        .withHandler(header0, testVersionHandler0)
+                        .withHandler(header12, testVersionHandler12)
                         .buildAndStart();
 
-        restClient =
-                new RestClient(
-                        config,
-                        Executors.newFixedThreadPool(
-                                1, new 
ExecutorThreadFactory("rest-client-thread-pool")));
+        restClient = getTestRestClient();
         serverAddress = serverEndpoint.getServerAddress();
     }
 
     @AfterEach
     void stop() throws Exception {
-
         if (restClient != null) {
-            restClient.shutdown(timeout);
+            restClient.shutdown();
             restClient = null;
         }
 
         if (serverEndpoint != null) {
-            serverEndpoint.closeAsync().get(timeout.getSize(), 
timeout.getUnit());
+            serverEndpoint.stop();
             serverEndpoint = null;
         }
     }
 
     /** Test that {@link SqlGatewayMessageHeaders} can identify the version 
correctly. */
     @Test
     void testSqlGatewayMessageHeaders() throws Exception {
-        // The header only support V1, but send request by V0
+        // The header can't support V0, but sends request by V0
         assertThatThrownBy(
                         () ->
                                 restClient.sendRequest(
                                         serverAddress.getHostName(),
                                         serverAddress.getPort(),
-                                        header2,
+                                        header12,
                                         EmptyMessageParameters.getInstance(),
                                         EmptyRequestBody.getInstance(),
                                         Collections.emptyList(),
                                         SqlGatewayRestAPIVersion.V0))
-                .isInstanceOf(IllegalArgumentException.class);
-
-        // The header only support V1, send request by V1
+                .satisfies(
+                        FlinkAssertions.anyCauseMatches(
+                                IllegalArgumentException.class,
+                                String.format(
+                                        "The requested version V0 is not 
supported by the request (method=%s URL=%s). Supported versions are: %s.",
+                                        header12.getHttpMethod(),
+                                        header12.getTargetRestEndpointURL(),
+                                        
header12.getSupportedAPIVersions().stream()
+                                                
.map(RestAPIVersion::getURLVersionPrefix)
+                                                
.collect(Collectors.joining(",")))));
+
+        // The header only supports V0, sends request by V0
         CompletableFuture<TestResponse> specifiedVersionResponse =
                 restClient.sendRequest(
                         serverAddress.getHostName(),
                         serverAddress.getPort(),
-                        header2,
+                        header0,
                         EmptyMessageParameters.getInstance(),
                         EmptyRequestBody.getInstance(),
                         Collections.emptyList(),
-                        SqlGatewayRestAPIVersion.V1);
+                        SqlGatewayRestAPIVersion.V0);
 
-        TestResponse testResponse1 = specifiedVersionResponse.get(5, 
TimeUnit.SECONDS);
-        assertThat(testResponse1.getStatus()).isEqualTo("V1");
+        TestResponse testResponse1 =
+                specifiedVersionResponse.get(timeout.getSize(), 
timeout.getUnit());
+        assertThat(testResponse1.getStatus()).isEqualTo("V0");
 
-        // The header only support V1, send request by latest version V1
+        // The header supports V1 and V2, lets the client get the latest 
version as default
         CompletableFuture<TestResponse> unspecifiedVersionResponse =
                 restClient.sendRequest(
                         serverAddress.getHostName(),
                         serverAddress.getPort(),
-                        header2,
+                        header12,
                         EmptyMessageParameters.getInstance(),
                         EmptyRequestBody.getInstance(),
                         Collections.emptyList());
 
-        TestResponse testResponse2 = unspecifiedVersionResponse.get(5, 
TimeUnit.SECONDS);
-        assertThat(testResponse2.getStatus()).isEqualTo("V1");
+        TestResponse testResponse2 =
+                unspecifiedVersionResponse.get(timeout.getSize(), 
timeout.getUnit());
+        assertThat(testResponse2.getStatus()).isEqualTo("V2");
     }
 
     /** Test that requests of different version are routed to correct 
handlers. */
     @Test
     void testVersionSelection() throws Exception {
-        CompletableFuture<TestResponse> version1Response =
-                restClient.sendRequest(
-                        serverAddress.getHostName(),
-                        serverAddress.getPort(),
-                        header1,
-                        EmptyMessageParameters.getInstance(),
-                        EmptyRequestBody.getInstance(),
-                        Collections.emptyList(),
-                        SqlGatewayRestAPIVersion.V0);
-
-        TestResponse testResponse = version1Response.get(5, TimeUnit.SECONDS);
-        assertThat(testResponse.getStatus()).isEqualTo("V0");
-
-        CompletableFuture<TestResponse> version2Response =
-                restClient.sendRequest(
-                        serverAddress.getHostName(),
-                        serverAddress.getPort(),
-                        header2,
-                        EmptyMessageParameters.getInstance(),
-                        EmptyRequestBody.getInstance(),
-                        Collections.emptyList(),
-                        SqlGatewayRestAPIVersion.V1);
-        TestResponse testResponse2 = version2Response.get(5, TimeUnit.SECONDS);
-        assertThat(testResponse2.getStatus()).isEqualTo("V1");
+        validateVersionSelection(header0, SqlGatewayRestAPIVersion.V0);
+        validateVersionSelection(header12, SqlGatewayRestAPIVersion.V1);
+        validateVersionSelection(header12, SqlGatewayRestAPIVersion.V2);

Review Comment:
   Use for-loop? In the future, we add a new version, we don't need to write 
another `validateVersionSelection`



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpointITCase.java:
##########
@@ -563,4 +502,51 @@ protected CompletableFuture<TestResponse> handleRequest(
             return CompletableFuture.completedFuture(new 
TestResponse(version.name()));
         }
     }
+
+    private static class TestBadCaseHandler
+            extends AbstractSqlGatewayRestHandler<
+                    TestRequest, TestResponse, EmptyMessageParameters> {
+
+        private final OneShotLatch closeLatch = new OneShotLatch();
+
+        private CompletableFuture<Void> closeFuture = 
CompletableFuture.completedFuture(null);
+
+        private Function<Integer, CompletableFuture<TestResponse>> handlerBody;
+
+        TestBadCaseHandler() {
+            super(null, Collections.emptyMap(), badCaseHeader);
+        }

Review Comment:
   I prefer to use the origin constructor.
   
   ```
    TestBadCaseHandler(SqlGatewayService sqlGatewayService) {                   
        Collections.emptyList(),
               super(sqlGatewayService, Collections.emptyMap(), badCaseHeader); 
                        version);
           }
   ```



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/util/SqlGatewayRestClientAndEndpointUtils.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.gateway.rest.util;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.rest.RestClient;
+import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.rest.SqlGatewayRestEndpoint;
+import org.apache.flink.util.ConfigurationException;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+/** The test REST client and endpoint utils for sql gateway rest api testing. 
*/
+public class SqlGatewayRestClientAndEndpointUtils {
+
+    // 
--------------------------------------------------------------------------------------------
+    // Rest Endpoint
+    // 
--------------------------------------------------------------------------------------------
+
+    /**
+     * Utility for setting up a rest server based on {@link 
SqlGatewayRestEndpoint} with a given set
+     * of handlers.
+     */
+    public static class TestSqlGatewayRestEndpoint extends 
SqlGatewayRestEndpoint {

Review Comment:
   Why merge the TestSqlGatewayRestEndpoint and TestClient into a util class? 
It's fine to introduce a two separated class. BTW, the class name 
`SqlGatewayRestClientAndEndpointUtils` is a little strange for others.... 



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpointITCase.java:
##########
@@ -532,27 +468,30 @@ public EmptyMessageParameters 
getUnresolvedMessageParameters() {
         }
     }
 
-    private static class TestVersionSelectionHeaders1 extends 
TestVersionSelectionHeadersBase {
+    private static class TestVersionSelectionHeaders0 extends 
TestVersionSelectionHeadersBase {
         @Override
         public Collection<SqlGatewayRestAPIVersion> getSupportedAPIVersions() {
             return Collections.singleton(SqlGatewayRestAPIVersion.V0);
         }
     }
 
-    private static class TestVersionSelectionHeaders2 extends 
TestVersionSelectionHeadersBase {
+    private static class TestVersionSelectionHeaders12 extends 
TestVersionSelectionHeadersBase {
         @Override
         public Collection<SqlGatewayRestAPIVersion> getSupportedAPIVersions() {
-            return Collections.singleton(SqlGatewayRestAPIVersion.V1);
+            return Arrays.asList(SqlGatewayRestAPIVersion.V1, 
SqlGatewayRestAPIVersion.V2);

Review Comment:
   ```
    List<SqlGatewayRestAPIVersion> versions =
                       Arrays.asList(SqlGatewayRestAPIVersion.values());
               versions.remove(SqlGatewayRestAPIVersion.V0);
               return versions;
   ```
   



##########
flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/SqlGatewayRestEndpointITCase.java:
##########
@@ -532,27 +468,30 @@ public EmptyMessageParameters 
getUnresolvedMessageParameters() {
         }
     }
 
-    private static class TestVersionSelectionHeaders1 extends 
TestVersionSelectionHeadersBase {
+    private static class TestVersionSelectionHeaders0 extends 
TestVersionSelectionHeadersBase {
         @Override
         public Collection<SqlGatewayRestAPIVersion> getSupportedAPIVersions() {
             return Collections.singleton(SqlGatewayRestAPIVersion.V0);
         }
     }
 
-    private static class TestVersionSelectionHeaders2 extends 
TestVersionSelectionHeadersBase {
+    private static class TestVersionSelectionHeaders12 extends 
TestVersionSelectionHeadersBase {

Review Comment:
   How about renaming to TestVersionSelectionHeadersNot0



-- 
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