pnowojski commented on a change in pull request #13:
URL: https://github.com/apache/flink-benchmarks/pull/13#discussion_r647449406



##########
File path: src/main/java/org/apache/flink/benchmark/InputBenchmark.java
##########
@@ -34,31 +35,33 @@
 
 @OperationsPerInvocation(value = InputBenchmark.RECORDS_PER_INVOCATION)
 public class InputBenchmark extends BenchmarkBase {
-
        public static final int RECORDS_PER_INVOCATION = 15_000_000;
        private static final long CHECKPOINT_INTERVAL_MS = 100;
 
        public static void main(String[] args)
-               throws RunnerException {
+                       throws RunnerException {
                Options options = new OptionsBuilder()
-                       .verbosity(VerboseMode.NORMAL)
-                       .include(".*" + InputBenchmark.class.getCanonicalName() 
+ ".*")
-                       .build();
+                               .verbosity(VerboseMode.NORMAL)
+                               .include(".*" + 
InputBenchmark.class.getCanonicalName() + ".*")
+                               .build();
 
                new Runner(options).run();
        }
 
+       @Param({"LEGACY_SOURCE", "FLIP_27_SOURCE_BOUNDED", 
"FLIP_27_SOURCE_UNBOUNDED"})

Review comment:
       This will become a part of the benchmark name, so I would shorten it as 
much as possible.
   
   ```
   {"LEGACY", "BOUNDED", "UNBOUNDED"}
   ```
   ```
   {"LEGACY", "F27_BOUNDED", "F27_UNBOUNDED"}
   ```
   ?

##########
File path: 
src/main/java/org/apache/flink/benchmark/functions/LongSourceType.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.benchmark.functions;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import java.util.function.BiFunction;
+
+/**
+ * Enum based factory for different Long sources.
+ */
+public enum LongSourceType {
+       LEGACY_SOURCE((env, maxValue) -> {
+               return env.addSource(new LongSource(maxValue));
+       }),
+       FLIP_27_SOURCE_BOUNDED((env, maxValue) -> {
+               return env.fromSource(
+                               new LongNewSource(Boundedness.BOUNDED, 
maxValue),
+                               WatermarkStrategy.noWatermarks(),
+                               "NewLongSource");
+       }),
+       FLIP_27_SOURCE_UNBOUNDED((env, maxValue) -> {
+               return env.fromSource(
+                               new LongNewSource(Boundedness.BOUNDED, 
maxValue),

Review comment:
       `FLIP_27_SOURCE_UNBOUNDED` & `BOUNDED`? How does it differ compared to 
`FLIP_27_SOURCE_BOUNDED`? Do we need both? Is there a runtime difference 
between bounded & unbounded?

##########
File path: src/main/java/org/apache/flink/benchmark/functions/LongNewSource.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.benchmark.functions;
+
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.lib.NumberSequenceSource;
+import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The source should produce same records as {@link LongSource}.
+ */
+public class LongNewSource extends NumberSequenceSource {
+       private final Boundedness boundedness;
+       private final long maxValue;
+
+       public LongNewSource(Boundedness boundedness, long maxValue) {
+               super(-1, -1); // we do not use the from/to of the underlying 
source

Review comment:
       What's wrong with `NumberSequenceSource`? Can not we use it as it is?




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


Reply via email to