Airblader commented on a change in pull request #16598:
URL: https://github.com/apache/flink/pull/16598#discussion_r680563823
##########
File path:
flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/table/PubSubTableSourceFactory.java
##########
@@ -0,0 +1,78 @@
+package org.apache.flink.streaming.connectors.gcp.pubsub.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DeserializationFormatFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+
+import java.util.HashSet;
+import java.util.Set;
+
+@Internal
+public class PubSubTableSourceFactory implements DynamicTableSourceFactory {
+
+ // define all options statically
+ public static final ConfigOption<String> PROJECT_NAME =
+ ConfigOptions.key("projectName").stringType().noDefaultValue();
+
+ public static final ConfigOption<String> TOPIC =
+ ConfigOptions.key("topic").stringType().noDefaultValue();
+
+ public static final ConfigOption<String> FORMAT =
+ ConfigOptions.key("format").stringType().noDefaultValue();
+
+ @Override
+ public String factoryIdentifier() {
+ return "pubsub";
+ }
+
+ @Override
+ public Set<ConfigOption<?>> requiredOptions() {
+ final Set<ConfigOption<?>> options = new HashSet<>();
+ options.add(PROJECT_NAME);
+ options.add(TOPIC);
+ options.add(FORMAT);
+ return options;
+ }
+
+ @Override
+ public Set<ConfigOption<?>> optionalOptions() {
+ return new HashSet<>();
+ }
+
+ @Override
+ public DynamicTableSource createDynamicTableSource(Context context) {
+ // either implement your custom validation logic here ...
+ // or use the provided helper utility
+ final FactoryUtil.TableFactoryHelper helper =
+ FactoryUtil.createTableFactoryHelper(this, context);
+
+ // validate all options
+ helper.validateExcept("json.");
Review comment:
Thanks for sharing that information. We cannot just exclude `json.*`
because if you use another format, the format options would also have a
different prefix.
The reason this fails is because we're currently calling things in the wrong
order. You can move the call to `helper.discoverDecodingFormat` to before (what
will now be) `helper.validate()`. This will register the options of the
discovered format internally so they're ignored when validating the connector
options.
--
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]