brachi-wernick commented on a change in pull request #16598:
URL: https://github.com/apache/flink/pull/16598#discussion_r680498203
##########
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:
The DDL:
```
CREATE TABLE `vvp`.`default`.`my_table` (
`user_id` VARCHAR(2147483647),
`country_code` VARCHAR(2147483647),
`event` VARCHAR(2147483647),
`os` VARCHAR(2147483647)
)
COMMENT ''
WITH (
'connector' = 'pubsub',
'format' = 'json',
'json.map-null-key.literal' = 'null',
'json.map-null-key.mode' = 'LITERAL',
'projectName' = 'myProj,
'topic' = 'mySubscription'
);
```
The error I get (if I don't exclude the 'json' from the validation):
```
Invalid connector configuration
The source connector for table 'vvp.default.my_table' could not be created.
Configured properties are: ['connector'='pubsub', 'format'='json',
'json.map-null-key.literal'='null', 'json.map-null-key.mode'='LITERAL',
'projectName'='myProj', 'topic'='mySubscription'].
Cause: Unsupported options found for 'pubsub'.
Unsupported options:
json.map-null-key.literal
json.map-null-key.mode
```
--
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]