WencongLiu commented on code in PR #20236:
URL: https://github.com/apache/flink/pull/20236#discussion_r921869944


##########
flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/handler/AbstractSqlGatewayRestHandler.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.handler;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
+import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
+import org.apache.flink.runtime.rest.handler.AbstractHandler;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.handler.util.HandlerUtils;
+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.runtime.rest.messages.ThreadDumpInfo;
+import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+import org.apache.flink.table.gateway.rest.util.RestEndpointVersion;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Super class for sql gateway handlers that work with {@link RequestBody}s 
and {@link
+ * ResponseBody}s.
+ *
+ * @param <R> type of incoming requests
+ * @param <P> type of outgoing responses
+ */
+public abstract class AbstractSqlGatewayRestHandler<
+                R extends RequestBody, P extends ResponseBody, M extends 
MessageParameters>
+        extends AbstractHandler<RestfulGateway, R, M> {
+
+    private static final EmptyLeaderAdapter EMPTY_LEADER_ADAPTER = new 
EmptyLeaderAdapter();
+    protected SqlGatewayService service;
+
+    private final MessageHeaders<R, P, M> messageHeaders;
+
+    protected AbstractSqlGatewayRestHandler(
+            SqlGatewayService service,
+            Time timeout,
+            Map<String, String> responseHeaders,
+            MessageHeaders<R, P, M> messageHeaders) {
+        super(
+                () -> CompletableFuture.completedFuture(EMPTY_LEADER_ADAPTER),
+                timeout,
+                responseHeaders,
+                messageHeaders);
+        this.service = service;
+        this.messageHeaders = messageHeaders;
+    }
+
+    @Override
+    protected CompletableFuture<Void> respondToRequest(
+            ChannelHandlerContext ctx,
+            HttpRequest httpRequest,
+            HandlerRequest<R> handlerRequest,
+            RestfulGateway gateway) {
+        CompletableFuture<P> response;
+
+        String uri = httpRequest.uri();
+        int slashIndex = uri.indexOf('/', 1);
+        if (slashIndex < 0) {
+            slashIndex = uri.length();
+        }
+
+        RestEndpointVersion version = 
RestEndpointVersion.valueOf(uri.substring(1, slashIndex));

Review Comment:
   The type of RestServerEndpoint is `enum`, so it can only implement 
interfaces. Therefore, it‘s hard to transfer version information through 
`Router#route` to `HandlerRequest`. 
   To solve this problem, I put the logic of converting uri to version 
in`SqlGatewayRestAPIVersion#fromURIToVersion`, and provide handling of parsing 
failures.



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