You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/05 12:07:38 UTC

[GitHub] [flink] wuchong commented on a diff in pull request #19849: [FLINK-27767][sql-gateway] Introduce Endpoint API and utils

wuchong commented on code in PR #19849:
URL: https://github.com/apache/flink/pull/19849#discussion_r913715882


##########
flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/endpoint/SqlGatewayEndpointFactoryUtils.java:
##########
@@ -0,0 +1,153 @@
+/*
+ *  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.api.endpoint;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DelegatingConfiguration;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.FactoryUtil.FactoryHelper;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** Util to discover the {@link SqlGatewayEndpoint}. */
+@PublicEvolving
+public class SqlGatewayEndpointFactoryUtils {
+
+    private static final String GATEWAY_ENDPOINT_PREFIX = "sql-gateway.endpoint";
+
+    public static final ConfigOption<List<String>> SQL_GATEWAY_ENDPOINT_TYPE =
+            ConfigOptions.key(String.format("%s.type", GATEWAY_ENDPOINT_PREFIX))
+                    .stringType()
+                    .asList()
+                    .noDefaultValue()
+                    .withDescription("Specify the endpoints that are used.");
+
+    /**
+     * Attempts to discover the appropriate endpoint factory and creates the instance of the
+     * endpoints.
+     */
+    public static List<SqlGatewayEndpoint> createSqlGatewayEndpoint(
+            SqlGatewayService service, Configuration configuration) {
+        List<String> identifiers = configuration.get(SQL_GATEWAY_ENDPOINT_TYPE);
+
+        if (identifiers == null || identifiers.isEmpty()) {
+            throw new ValidationException(
+                    String.format(
+                            "Endpoint options do not contain an option key '%s' for discovering an endpoint.",
+                            SQL_GATEWAY_ENDPOINT_TYPE.key()));
+        }
+        validateSpecifiedEndpointsAreUnique(identifiers);
+
+        List<SqlGatewayEndpoint> endpoints = new ArrayList<>();
+        for (String identifier : identifiers) {
+            final SqlGatewayEndpointFactory factory =
+                    FactoryUtil.discoverFactory(
+                            Thread.currentThread().getContextClassLoader(),
+                            SqlGatewayEndpointFactory.class,
+                            identifier);
+            Configuration endpointConfig =
+                    new DelegatingConfiguration(
+                            configuration,
+                            String.format("%s.%s.", GATEWAY_ENDPOINT_PREFIX, identifier));
+            endpoints.add(
+                    factory.createSqlGatewayEndpoint(
+                            new DefaultSqlGatewayEndpointFactoryContext(service, endpointConfig)));
+        }
+        return endpoints;
+    }
+
+    /**
+     * Creates a utility that helps to validate options for a {@link SqlGatewayEndpointFactory}.
+     *
+     * <p>Note: This utility checks for left-over options in the final step.
+     */
+    public static SqlGatewayEndpointFactoryHelper createSqlGatewayEndpointFactoryHelper(

Review Comment:
   nit: Maybe we can simplify the name to `EndpointFactoryHelper`.



##########
flink-table/flink-sql-gateway-api/src/main/java/org/apache/flink/table/gateway/api/endpoint/SqlGatewayEndpointFactoryUtils.java:
##########
@@ -0,0 +1,153 @@
+/*
+ *  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.api.endpoint;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DelegatingConfiguration;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.factories.FactoryUtil.FactoryHelper;
+import org.apache.flink.table.gateway.api.SqlGatewayService;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/** Util to discover the {@link SqlGatewayEndpoint}. */
+@PublicEvolving
+public class SqlGatewayEndpointFactoryUtils {
+
+    private static final String GATEWAY_ENDPOINT_PREFIX = "sql-gateway.endpoint";
+
+    public static final ConfigOption<List<String>> SQL_GATEWAY_ENDPOINT_TYPE =
+            ConfigOptions.key(String.format("%s.type", GATEWAY_ENDPOINT_PREFIX))
+                    .stringType()
+                    .asList()
+                    .noDefaultValue()
+                    .withDescription("Specify the endpoints that are used.");
+
+    /**
+     * Attempts to discover the appropriate endpoint factory and creates the instance of the
+     * endpoints.
+     */
+    public static List<SqlGatewayEndpoint> createSqlGatewayEndpoint(
+            SqlGatewayService service, Configuration configuration) {
+        List<String> identifiers = configuration.get(SQL_GATEWAY_ENDPOINT_TYPE);
+
+        if (identifiers == null || identifiers.isEmpty()) {
+            throw new ValidationException(
+                    String.format(
+                            "Endpoint options do not contain an option key '%s' for discovering an endpoint.",
+                            SQL_GATEWAY_ENDPOINT_TYPE.key()));
+        }
+        validateSpecifiedEndpointsAreUnique(identifiers);
+
+        List<SqlGatewayEndpoint> endpoints = new ArrayList<>();
+        for (String identifier : identifiers) {
+            final SqlGatewayEndpointFactory factory =
+                    FactoryUtil.discoverFactory(
+                            Thread.currentThread().getContextClassLoader(),
+                            SqlGatewayEndpointFactory.class,
+                            identifier);
+            Configuration endpointConfig =
+                    new DelegatingConfiguration(
+                            configuration,
+                            String.format("%s.%s.", GATEWAY_ENDPOINT_PREFIX, identifier));
+            endpoints.add(
+                    factory.createSqlGatewayEndpoint(
+                            new DefaultSqlGatewayEndpointFactoryContext(service, endpointConfig)));
+        }
+        return endpoints;
+    }
+
+    /**
+     * Creates a utility that helps to validate options for a {@link SqlGatewayEndpointFactory}.
+     *
+     * <p>Note: This utility checks for left-over options in the final step.
+     */
+    public static SqlGatewayEndpointFactoryHelper createSqlGatewayEndpointFactoryHelper(
+            SqlGatewayEndpointFactory endpointFactory, SqlGatewayEndpointFactory.Context context) {
+        return new SqlGatewayEndpointFactoryHelper(endpointFactory, context.getOptions());
+    }
+
+    // ----------------------------------------------------------------------------------------
+
+    /**
+     * Helper utility for validating all options for a {@link SqlGatewayEndpointFactory}.
+     *
+     * @see #createSqlGatewayEndpointFactoryHelper(SqlGatewayEndpointFactory,
+     *     SqlGatewayEndpointFactory.Context)
+     */
+    public static class SqlGatewayEndpointFactoryHelper
+            extends FactoryHelper<SqlGatewayEndpointFactory> {
+
+        private SqlGatewayEndpointFactoryHelper(
+                SqlGatewayEndpointFactory factory, Map<String, String> configOptions) {
+            super(factory, configOptions);
+        }
+    }
+
+    private static class DefaultSqlGatewayEndpointFactoryContext
+            implements SqlGatewayEndpointFactory.Context {
+
+        private final SqlGatewayService service;
+        private final Configuration configuration;
+
+        public DefaultSqlGatewayEndpointFactoryContext(
+                SqlGatewayService service, Configuration configuration) {
+            this.service = service;
+            this.configuration = configuration;
+        }
+
+        @Override
+        public SqlGatewayService getSqlGatewayService() {
+            return service;
+        }
+
+        @Override
+        public ReadableConfig getConfiguration() {
+            return configuration;
+        }
+
+        @Override
+        public Map<String, String> getOptions() {
+            return configuration.toMap();
+        }
+    }
+
+    private static void validateSpecifiedEndpointsAreUnique(List<String> identifiers) {
+        Set<String> uniqueIdentifiers = new HashSet<>();
+
+        for (String identifier : identifiers) {
+            if (uniqueIdentifiers.contains(identifier)) {
+                throw new ValidationException(
+                        String.format(
+                                "Get the duplicate endpoint identifier: %s. Please keep the specified endpoint identifier unique.",

Review Comment:
   nit: please also mention the configuration in the exception message, otherwise, it's hard for users to figure out what's wrong. 



##########
flink-table/flink-sql-gateway-api/pom.xml:
##########
@@ -46,6 +46,11 @@
             <artifactId>flink-table-api-java-bridge</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-test-utils-junit</artifactId>
+            <version>${project.version}</version>

Review Comment:
   test scope is enough for the module. 



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org