lirui-apache commented on a change in pull request #12864:
URL: https://github.com/apache/flink/pull/12864#discussion_r457861642



##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/DataGenTableSourceFactory.java
##########
@@ -94,128 +97,176 @@ public String factoryIdentifier() {
 
        @Override
        public DynamicTableSource createDynamicTableSource(Context context) {
+               createTableFactoryHelper(this, context).validateExcept(FIELDS);

Review comment:
       Is this necessary, given that we'll call 
`FactoryUtil.validateFactoryOptions` and `FactoryUtil.validateUnconsumedKeys` 
later on?

##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/DataGenTableSourceFactory.java
##########
@@ -94,128 +97,176 @@ public String factoryIdentifier() {
 
        @Override
        public DynamicTableSource createDynamicTableSource(Context context) {
+               createTableFactoryHelper(this, context).validateExcept(FIELDS);
+
                Configuration options = new Configuration();
                
context.getCatalogTable().getOptions().forEach(options::setString);
 
-               TableSchema tableSchema = 
TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+               TableSchema schema = 
TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+               DataGenerator[] fieldGenerators = new 
DataGenerator[schema.getFieldCount()];
+               Set<ConfigOption<?>> optionalOptions = new HashSet<>();
 
-               DataGenerator[] fieldGenerators = new 
DataGenerator[tableSchema.getFieldCount()];
                for (int i = 0; i < fieldGenerators.length; i++) {
-                       fieldGenerators[i] = createDataGenerator(
-                                       tableSchema.getFieldName(i).get(),
-                                       tableSchema.getFieldDataType(i).get(),
-                                       options);
+                       String name = schema.getFieldNames()[i];
+                       DataType type = schema.getFieldDataTypes()[i];
+
+                       ConfigOption<String> kind = key(FIELDS + "." + name + 
"." + KIND)
+                                       .stringType().defaultValue(RANDOM);
+                       DataGeneratorContainer container = 
createContainer(name, type, options.get(kind), options);
+                       fieldGenerators[i] = container.generator;
+
+                       optionalOptions.add(kind);
+                       optionalOptions.addAll(container.options);
                }
 
-               return new DataGenTableSource(fieldGenerators, tableSchema, 
options.get(ROWS_PER_SECOND));
+               FactoryUtil.validateFactoryOptions(new HashSet<>(), 
optionalOptions, options);
+
+               Set<String> consumedOptionKeys = new HashSet<>();
+               consumedOptionKeys.add(CONNECTOR.key());
+               consumedOptionKeys.add(ROWS_PER_SECOND.key());
+               
optionalOptions.stream().map(ConfigOption::key).forEach(consumedOptionKeys::add);
+               FactoryUtil.validateUnconsumedKeys(factoryIdentifier(), 
options.keySet(), consumedOptionKeys);
+
+               return new DataGenTableSource(fieldGenerators, schema, 
options.get(ROWS_PER_SECOND));
        }
 
-       private DataGenerator createDataGenerator(String name, DataType type, 
ReadableConfig options) {
-               String genType = options.get(
-                               key(FIELDS + "." + name + "." + 
KIND).stringType().defaultValue(RANDOM));
-               switch (genType) {
+       private DataGeneratorContainer createContainer(
+                       String name, DataType type, String kind, ReadableConfig 
options) {
+               switch (kind) {
                        case RANDOM:
-                               return createRandomGenerator(name, type, 
options);
+                               return createRandomContainer(name, type, 
options);
                        case SEQUENCE:
-                               return createSequenceGenerator(name, type, 
options);
+                               return createSequenceContainer(name, type, 
options);
                        default:
-                               throw new ValidationException("Unsupported 
generator type: " + genType);
+                               throw new ValidationException("Unsupported 
generator kind: " + kind);
                }
        }
 
-       private DataGenerator createRandomGenerator(String name, DataType type, 
ReadableConfig options) {
-               ConfigOption<Integer> lenKey = key(FIELDS + "." + name + "." + 
LENGTH)
-                               .intType().defaultValue(100);
+       private DataGeneratorContainer createRandomContainer(String name, 
DataType type, ReadableConfig config) {
                OptionBuilder minKey = key(FIELDS + "." + name + "." + MIN);
                OptionBuilder maxKey = key(FIELDS + "." + name + "." + MAX);
                switch (type.getLogicalType().getTypeRoot()) {
-                       case BOOLEAN:
-                               return RandomGenerator.booleanGenerator();
+                       case BOOLEAN: {
+                               return 
DataGeneratorContainer.of(RandomGenerator.booleanGenerator());
+                       }
                        case CHAR:
-                       case VARCHAR:
-                               int length = options.get(lenKey);
-                               return getRandomStringGenerator(length);
-                       case TINYINT:
-                               return RandomGenerator.byteGenerator(
-                                               
options.get(minKey.intType().defaultValue((int) Byte.MIN_VALUE)).byteValue(),
-                                               
options.get(maxKey.intType().defaultValue((int) Byte.MAX_VALUE)).byteValue());
-                       case SMALLINT:
-                               return RandomGenerator.shortGenerator(
-                                               
options.get(minKey.intType().defaultValue((int) Short.MIN_VALUE)).shortValue(),
-                                               
options.get(maxKey.intType().defaultValue((int) Short.MAX_VALUE)).shortValue());
-                       case INTEGER:
-                               return RandomGenerator.intGenerator(
-                                               
options.get(minKey.intType().defaultValue(Integer.MIN_VALUE)),
-                                               
options.get(maxKey.intType().defaultValue(Integer.MAX_VALUE)));
-                       case BIGINT:
-                               return RandomGenerator.longGenerator(
-                                               
options.get(minKey.longType().defaultValue(Long.MIN_VALUE)),
-                                               
options.get(maxKey.longType().defaultValue(Long.MAX_VALUE)));
-                       case FLOAT:
-                               return RandomGenerator.floatGenerator(
-                                               
options.get(minKey.floatType().defaultValue(Float.MIN_VALUE)),
-                                               
options.get(maxKey.floatType().defaultValue(Float.MAX_VALUE)));
-                       case DOUBLE:
-                               return RandomGenerator.doubleGenerator(
-                                               
options.get(minKey.doubleType().defaultValue(Double.MIN_VALUE)),
-                                               
options.get(maxKey.doubleType().defaultValue(Double.MAX_VALUE)));
+                       case VARCHAR: {
+                               ConfigOption<Integer> lenOption = key(FIELDS + 
"." + name + "." + LENGTH)
+                                               .intType()
+                                               .defaultValue(100);

Review comment:
       make the default value a static member?

##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/DataGenTableSourceFactory.java
##########
@@ -94,128 +97,176 @@ public String factoryIdentifier() {
 
        @Override
        public DynamicTableSource createDynamicTableSource(Context context) {
+               createTableFactoryHelper(this, context).validateExcept(FIELDS);
+
                Configuration options = new Configuration();
                
context.getCatalogTable().getOptions().forEach(options::setString);
 
-               TableSchema tableSchema = 
TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+               TableSchema schema = 
TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+               DataGenerator[] fieldGenerators = new 
DataGenerator[schema.getFieldCount()];
+               Set<ConfigOption<?>> optionalOptions = new HashSet<>();
 
-               DataGenerator[] fieldGenerators = new 
DataGenerator[tableSchema.getFieldCount()];
                for (int i = 0; i < fieldGenerators.length; i++) {
-                       fieldGenerators[i] = createDataGenerator(
-                                       tableSchema.getFieldName(i).get(),
-                                       tableSchema.getFieldDataType(i).get(),
-                                       options);
+                       String name = schema.getFieldNames()[i];
+                       DataType type = schema.getFieldDataTypes()[i];
+
+                       ConfigOption<String> kind = key(FIELDS + "." + name + 
"." + KIND)
+                                       .stringType().defaultValue(RANDOM);
+                       DataGeneratorContainer container = 
createContainer(name, type, options.get(kind), options);
+                       fieldGenerators[i] = container.generator;
+
+                       optionalOptions.add(kind);
+                       optionalOptions.addAll(container.options);
                }
 
-               return new DataGenTableSource(fieldGenerators, tableSchema, 
options.get(ROWS_PER_SECOND));
+               FactoryUtil.validateFactoryOptions(new HashSet<>(), 
optionalOptions, options);
+
+               Set<String> consumedOptionKeys = new HashSet<>();
+               consumedOptionKeys.add(CONNECTOR.key());
+               consumedOptionKeys.add(ROWS_PER_SECOND.key());

Review comment:
       Do we also need to add `PROPERTY_VERSION` to `consumedOptionKeys`?

##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/DataGenTableSourceFactory.java
##########
@@ -94,128 +97,176 @@ public String factoryIdentifier() {
 
        @Override
        public DynamicTableSource createDynamicTableSource(Context context) {
+               createTableFactoryHelper(this, context).validateExcept(FIELDS);
+
                Configuration options = new Configuration();
                
context.getCatalogTable().getOptions().forEach(options::setString);
 
-               TableSchema tableSchema = 
TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+               TableSchema schema = 
TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
+               DataGenerator[] fieldGenerators = new 
DataGenerator[schema.getFieldCount()];
+               Set<ConfigOption<?>> optionalOptions = new HashSet<>();
 
-               DataGenerator[] fieldGenerators = new 
DataGenerator[tableSchema.getFieldCount()];
                for (int i = 0; i < fieldGenerators.length; i++) {
-                       fieldGenerators[i] = createDataGenerator(
-                                       tableSchema.getFieldName(i).get(),
-                                       tableSchema.getFieldDataType(i).get(),
-                                       options);
+                       String name = schema.getFieldNames()[i];
+                       DataType type = schema.getFieldDataTypes()[i];
+
+                       ConfigOption<String> kind = key(FIELDS + "." + name + 
"." + KIND)
+                                       .stringType().defaultValue(RANDOM);
+                       DataGeneratorContainer container = 
createContainer(name, type, options.get(kind), options);
+                       fieldGenerators[i] = container.generator;
+
+                       optionalOptions.add(kind);
+                       optionalOptions.addAll(container.options);
                }
 
-               return new DataGenTableSource(fieldGenerators, tableSchema, 
options.get(ROWS_PER_SECOND));
+               FactoryUtil.validateFactoryOptions(new HashSet<>(), 
optionalOptions, options);

Review comment:
       Call `requiredOptions()` instead of `new HashSet<>()`?




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

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


Reply via email to