JingsongLi commented on a change in pull request #13010:
URL: https://github.com/apache/flink/pull/13010#discussion_r484163628



##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/datagen/DataGenVisitorBase.java
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.factories.datagen;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.source.datagen.DataGenerator;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.ZonedTimestampType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor;
+
+import java.io.Serializable;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.util.function.Supplier;
+
+/**
+ * Base class for translating {@link LogicalType LogicalTypes} to {@link 
DataGeneratorContainer}'s.
+ */
+public abstract class DataGenVisitorBase extends 
LogicalTypeDefaultVisitor<DataGeneratorContainer> {
+
+       protected final String name;
+
+       protected final ReadableConfig config;
+
+       protected DataGenVisitorBase(String name, ReadableConfig config) {
+               this.name = name;
+               this.config = config;
+       }
+
+       @Override
+       public DataGeneratorContainer visit(DateType dateType) {
+               return 
DataGeneratorContainer.of(TimeGenerator.of(LocalDate::now));
+       }
+
+       @Override
+       public DataGeneratorContainer visit(TimeType timeType) {
+               return 
DataGeneratorContainer.of(TimeGenerator.of(LocalTime::now));
+       }
+
+       @Override
+       public DataGeneratorContainer visit(TimestampType timestampType) {
+               return 
DataGeneratorContainer.of(TimeGenerator.of(LocalDateTime::now));
+       }
+
+       @Override
+       public DataGeneratorContainer visit(ZonedTimestampType 
zonedTimestampType) {
+               return 
DataGeneratorContainer.of(TimeGenerator.of(OffsetDateTime::now));
+       }
+
+       @Override
+       public DataGeneratorContainer visit(LocalZonedTimestampType 
localZonedTimestampType) {
+               return 
DataGeneratorContainer.of(TimeGenerator.of(Instant::now));
+       }
+
+       @Override
+       protected DataGeneratorContainer defaultMethod(LogicalType logicalType) 
{
+               throw new ValidationException("Unsupported type: " + 
logicalType);
+       }
+
+       private interface SerializableSupplier<T> extends Supplier<T>, 
Serializable { }

Review comment:
       I think lambda is already `Serializable`.

##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/datagen/SequenceGeneratorVisitor.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.factories.datagen;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.functions.source.datagen.RandomGenerator;
+import 
org.apache.flink.streaming.api.functions.source.datagen.SequenceGenerator;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+import static org.apache.flink.table.factories.DataGenTableSourceFactory.END;
+import static 
org.apache.flink.table.factories.DataGenTableSourceFactory.FIELDS;
+import static org.apache.flink.table.factories.DataGenTableSourceFactory.START;
+
+/**
+ * Creates a sequential {@link DataGeneratorContainer} for a particular 
logical type.
+ */
+@Internal
+public class SequenceGeneratorVisitor extends DataGenVisitorBase {
+
+       private final ReadableConfig config;
+
+       private final String startKeyStr;
+
+       private final String endKeyStr;
+
+       private final ConfigOption<Integer> intStart;
+
+       private final ConfigOption<Integer> intEnd;
+
+       private final ConfigOption<Long> longStart;
+
+       private final ConfigOption<Long> longEnd;
+
+       public SequenceGeneratorVisitor(String name, ReadableConfig config) {
+               super(name, config);
+
+               this.config = config;
+
+               this.startKeyStr = FIELDS + "." + name + "." + START;
+               this.endKeyStr = FIELDS + "." + name + "." + END;
+
+               ConfigOptions.OptionBuilder startKey = key(startKeyStr);
+               ConfigOptions.OptionBuilder endKey = key(endKeyStr);
+
+               
config.getOptional(startKey.stringType().noDefaultValue()).orElseThrow(
+                       () -> new ValidationException(
+                               "Could not find required property '" + 
startKeyStr + "' for sequence generator."));
+               
config.getOptional(endKey.stringType().noDefaultValue()).orElseThrow(
+                       () -> new ValidationException(
+                               "Could not find required property '" + 
endKeyStr + "' for sequence generator."));
+
+               this.intStart = startKey.intType().noDefaultValue();
+               this.intEnd = endKey.intType().noDefaultValue();
+               this.longStart = startKey.longType().noDefaultValue();
+               this.longEnd = endKey.longType().noDefaultValue();
+       }
+
+       @Override
+       public DataGeneratorContainer visit(BooleanType booleanType) {
+               return 
DataGeneratorContainer.of(RandomGenerator.booleanGenerator());
+       }
+
+       @Override
+       public DataGeneratorContainer visit(CharType booleanType) {
+               return DataGeneratorContainer.of(
+                       getSequenceStringGenerator(
+                               config.get(longStart), config.get(longEnd)),
+                       longStart, longEnd);
+       }
+
+       @Override
+       public DataGeneratorContainer visit(VarCharType booleanType) {
+               return DataGeneratorContainer.of(
+                       getSequenceStringGenerator(
+                               config.get(longStart), config.get(longEnd)),
+                       longStart, longEnd);
+       }
+
+       @Override
+       public DataGeneratorContainer visit(TinyIntType booleanType) {
+               return DataGeneratorContainer.of(
+                       SequenceGenerator.byteGenerator(
+                               config.get(intStart).byteValue(),
+                               config.get(intEnd).byteValue()),
+                       intStart, intEnd);
+       }
+
+       @Override
+       public DataGeneratorContainer visit(SmallIntType booleanType) {
+               return DataGeneratorContainer.of(
+                       SequenceGenerator.shortGenerator(
+                               config.get(intStart).shortValue(),
+                               config.get(intEnd).shortValue()),
+                       intStart, intEnd);
+       }
+
+       @Override
+       public DataGeneratorContainer visit(IntType integerType) {
+               return DataGeneratorContainer.of(
+                       SequenceGenerator.intGenerator(
+                               config.get(intStart), config.get(intEnd)),
+                       intStart, intEnd);
+       }
+
+       @Override
+       public DataGeneratorContainer visit(BigIntType bigIntType) {
+               return DataGeneratorContainer.of(
+                       SequenceGenerator.longGenerator(
+                               config.get(longStart), config.get(longEnd)),
+                       longStart, longEnd);
+       }
+
+       @Override
+       public DataGeneratorContainer visit(FloatType floatType) {
+               return DataGeneratorContainer.of(
+                       SequenceGenerator.floatGenerator(
+                               config.get(intStart).shortValue(),
+                               config.get(intEnd).shortValue()),
+                       intStart, intEnd);
+       }
+
+       @Override
+       public DataGeneratorContainer visit(DoubleType doubleType) {
+               return DataGeneratorContainer.of(
+                       SequenceGenerator.doubleGenerator(
+                               config.get(intStart), config.get(intEnd)),
+                       intStart, intEnd);
+       }
+
+       @Override
+       public DataGeneratorContainer visit(DecimalType decimalType) {
+               return DataGeneratorContainer.of(
+                       SequenceGenerator.bigDecimalGenerator(

Review comment:
       Looks like the structures of Decimal, Times are wrong, you can take a 
look to the comments of `RowData`.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/DataGeneratorSource.java
##########
@@ -45,18 +62,34 @@
         * @param generator data generator.
         */
        public DataGeneratorSource(DataGenerator<T> generator) {
-               this(generator, Long.MAX_VALUE);
+               this(generator, "generator", Long.MAX_VALUE, null);
        }
 
        /**
         * Creates a source that emits records by {@link DataGenerator}.
         *
         * @param generator data generator.
         * @param rowsPerSecond Control the emit rate.
+        * @param numberOfRows Total number of rows to output.
         */
-       public DataGeneratorSource(DataGenerator<T> generator, long 
rowsPerSecond) {
+       public DataGeneratorSource(DataGenerator<T> generator, String name, 
long rowsPerSecond, Long numberOfRows) {
                this.generator = generator;
+               this.name = name;

Review comment:
       Do we need to add name?

##########
File path: 
flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/DataGeneratorConnectorITCase.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.planner.runtime.stream.table;
+
+import org.apache.flink.table.planner.runtime.utils.StreamingTestBase;
+
+import org.junit.Test;
+
+public class DataGeneratorConnectorITCase extends StreamingTestBase {

Review comment:
       You can use `BatchTestBase`.

##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/datagen/types/DataGeneratorMapper.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.factories.datagen.types;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.streaming.api.functions.source.datagen.DataGenerator;
+
+import java.io.Serializable;
+import java.util.function.Function;
+
+/**
+ * Utility for mapping the output of a {@link DataGenerator}.
+ */
+@Internal
+public class DataGeneratorMapper<A, B> implements DataGenerator<B> {
+
+       private final DataGenerator<A> generator;
+
+       private final SerializableFunction<A, B> mapper;
+
+       public DataGeneratorMapper(DataGenerator<A> generator, 
SerializableFunction<A, B> mapper) {
+               this.generator = generator;
+               this.mapper = mapper;
+       }
+
+       @Override
+       public void open(String name, FunctionInitializationContext context, 
RuntimeContext runtimeContext) throws Exception {
+               generator.open(name, context, runtimeContext);
+       }
+
+       @Override
+       public boolean hasNext() {
+               return generator.hasNext();
+       }
+
+       @Override
+       public B next() {
+               return mapper.apply(generator.next());
+       }
+
+       public interface SerializableFunction<A, B> extends Function<A, B>, 
Serializable {}

Review comment:
       Ditto.

##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/datagen/DataGenTableSource.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.factories.datagen;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
+import org.apache.flink.streaming.api.functions.source.datagen.DataGenerator;
+import 
org.apache.flink.streaming.api.functions.source.datagen.DataGeneratorSource;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.datagen.types.RowDataGenerator;
+import org.apache.flink.table.sources.StreamTableSource;
+
+/**
+ * A {@link StreamTableSource} that emits each number from a given interval 
exactly once,
+ * possibly in parallel. See {@link StatefulSequenceSource}.
+ */
+@Internal
+public class DataGenTableSource implements ScanTableSource {
+
+       private final DataGenerator[] fieldGenerators;
+       private final String tableName;
+       private final TableSchema schema;
+       private final long rowsPerSecond;
+       private final Long numberOfRows;
+
+       public DataGenTableSource(
+               DataGenerator[] fieldGenerators,
+               String tableName,
+               TableSchema schema,
+               long rowsPerSecond,
+               Long numberOfRows) {
+               this.fieldGenerators = fieldGenerators;
+               this.tableName = tableName;
+               this.schema = schema;
+               this.rowsPerSecond = rowsPerSecond;
+               this.numberOfRows = numberOfRows;
+       }
+
+       @Override
+       public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) {
+               boolean isBounded = numberOfRows == null;

Review comment:
       Looks like the opposite?

##########
File path: 
flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/factories/datagen/types/RowDataGenerator.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.factories.datagen.types;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.functions.source.datagen.DataGenerator;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+
+/**
+ * Data generator for Flink's internal {@link RowData} type.
+ */
+@Internal
+public class RowDataGenerator implements DataGenerator<RowData> {
+
+       private static final long serialVersionUID = 1L;
+
+       private final DataGenerator[] fieldGenerators;

Review comment:
       Can we add `?` to `DataGenerator` like `DataGenerator<?>[]`? At least, 
The compiler will have no warning.

##########
File path: 
flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/DataGeneratorConnectorITCase.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.planner.runtime.stream.table;
+
+import org.apache.flink.table.planner.runtime.utils.StreamingTestBase;
+
+import org.junit.Test;
+
+public class DataGeneratorConnectorITCase extends StreamingTestBase {
+
+       private static final String TABLE = "CREATE TABLE datagen_t (\n" +
+               "       f0 CHAR(1),\n" +
+               "       f1 VARCHAR(10),\n" +
+               "       f2 STRING,\n" +
+               "       f3 BOOLEAN,\n" +
+               "       f4 DECIMAL(32,2),\n" +
+               "       f5 TINYINT,\n" +
+               "       f6 SMALLINT,\n" +
+               "       f7 INT,\n" +
+               "       f8 BIGINT,\n" +
+               "       f9 FLOAT,\n" +
+               "       f10 DOUBLE,\n" +
+               "       f11 DATE,\n" +
+               "       f12 TIME,\n" +
+               "       f13 TIMESTAMP(3),\n" +
+               "       f14 TIMESTAMP WITH LOCAL TIME ZONE,\n" +
+               "       f15 INT ARRAY,\n" +
+               "       f16 MAP<STRING, DATE>,\n" +
+               "       f17 DECIMAL(32,2) MULTISET,\n" +
+               "       f18 ROW<a BIGINT, b TIME, c ROW<d TIMESTAMP>>\n" +
+               ") WITH (" +
+               "       'connector' = 'datagen',\n" +
+               "       'number-of-rows' = '10'\n" +
+               ")";
+
+       private static final String SINK = "CREATE TABLE sink WITH ('connector' 
= 'blackhole') LIKE datagen_t (EXCLUDING ALL)";
+
+       @Test
+       public void testTypes() {
+               tEnv().executeSql(TABLE);
+               tEnv().executeSql(SINK);
+               tEnv().from("datagen_t").executeInsert("sink");

Review comment:
       You should use: `Lists.newArrayList(tEnv().executeSql("select * from 
datagen_t").collect())`.
   In this way, the types can be verified.




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