bowenli86 commented on a change in pull request #11496: URL: https://github.com/apache/flink/pull/11496#discussion_r414739195
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/DataGeneratorSource.java ########## @@ -0,0 +1,102 @@ +/* + * 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.streaming.api.functions.source.datagen; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; + +/** + * A data generator source that abstract data generator. It can used to easy startup/test + * for streaming job and performance testing. + * It is stateful, re-scalable, possibly in parallel. + */ +@Experimental +public class DataGeneratorSource<T> extends RichParallelSourceFunction<T> implements CheckpointedFunction { + + private static final long serialVersionUID = 1L; + + private final DataGenerator<T> generator; + private final long rowsPerSecond; + + transient volatile boolean isRunning; + + /** + * Creates a source that emits records by {@link DataGenerator} without controlling emit rate. + * + * @param generator data generator. + */ + public DataGeneratorSource(DataGenerator<T> generator) { + this(generator, Long.MAX_VALUE); + } + + /** + * Creates a source that emits records by {@link DataGenerator}. + * + * @param generator data generator. + * @param rowsPerSecond Control the emit rate. + */ + public DataGeneratorSource(DataGenerator<T> generator, long rowsPerSecond) { + this.generator = generator; + this.rowsPerSecond = rowsPerSecond; + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + this.generator.open("TOP_FIELD", context, getRuntimeContext()); Review comment: where does the "TOP_FIELD" come from? ########## File path: flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/BlackHoleTableSinkFactory.java ########## @@ -0,0 +1,93 @@ +/* + * 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.sinks; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.factories.TableSinkFactory; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.DescriptorProperties.TABLE_SCHEMA_EXPR; +import static org.apache.flink.table.descriptors.Schema.SCHEMA; + +/** + * Factory base for creating configured instances of {@link CsvTableSink} in a stream environment. Review comment: update comment? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/RandomGenerator.java ########## @@ -0,0 +1,119 @@ +/* + * 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.streaming.api.functions.source.datagen; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.runtime.state.FunctionInitializationContext; + +import org.apache.commons.math3.random.RandomDataGenerator; + +/** + * Random generator. + */ +@Experimental +public abstract class RandomGenerator<T> implements DataGenerator<T> { + + protected transient RandomDataGenerator random; + + @Override + public void open( + String name, + FunctionInitializationContext context, + RuntimeContext runtimeContext) throws Exception { + this.random = new RandomDataGenerator(); + } + + @Override + public boolean hasNext() { + return true; + } + Review comment: a few small feature requests: add decimal, and an array generator with the existing primitive types? ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/DataGeneratorSource.java ########## @@ -0,0 +1,102 @@ +/* + * 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.streaming.api.functions.source.datagen; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; + +/** + * A data generator source that abstract data generator. It can used to easy startup/test + * for streaming job and performance testing. + * It is stateful, re-scalable, possibly in parallel. + */ +@Experimental +public class DataGeneratorSource<T> extends RichParallelSourceFunction<T> implements CheckpointedFunction { + + private static final long serialVersionUID = 1L; + + private final DataGenerator<T> generator; + private final long rowsPerSecond; + + transient volatile boolean isRunning; + + /** + * Creates a source that emits records by {@link DataGenerator} without controlling emit rate. + * + * @param generator data generator. + */ + public DataGeneratorSource(DataGenerator<T> generator) { + this(generator, Long.MAX_VALUE); + } + + /** + * Creates a source that emits records by {@link DataGenerator}. + * + * @param generator data generator. + * @param rowsPerSecond Control the emit rate. + */ + public DataGeneratorSource(DataGenerator<T> generator, long rowsPerSecond) { + this.generator = generator; + this.rowsPerSecond = rowsPerSecond; + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + this.generator.open("TOP_FIELD", context, getRuntimeContext()); + this.isRunning = true; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + this.generator.snapshotState(context); + } + + @Override + public void run(SourceContext<T> ctx) throws Exception { + double taskRowsPerSecond = (double) rowsPerSecond / getRuntimeContext().getNumberOfParallelSubtasks(); + long readTimeIncrement = (long) (1000 / taskRowsPerSecond); + long nextReadTime = System.currentTimeMillis(); + + while (isRunning && generator.hasNext()) { + synchronized (ctx.getCheckpointLock()) { + ctx.collect(this.generator.next()); + } + + if (readTimeIncrement > 0) { Review comment: what if I want 10,000 and 20,000 rps with single parallelism? How does the rate control work? ########## File path: flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/PrintTableSinkFactory.java ########## @@ -0,0 +1,94 @@ +/* + * 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.sinks; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.factories.TableSinkFactory; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.DescriptorProperties.TABLE_SCHEMA_EXPR; +import static org.apache.flink.table.descriptors.Schema.SCHEMA; + +/** + * Factory base for creating configured instances of {@link CsvTableSink} in a stream environment. Review comment: update comment? ---------------------------------------------------------------- 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: [email protected]
