Myasuka commented on code in PR #83:
URL: https://github.com/apache/flink-benchmarks/pull/83#discussion_r1429199281


##########
src/main/java/org/apache/flink/state/benchmark/ttl/TtlListStateBenchmark.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.state.benchmark.ttl;
+
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend;
+import org.apache.flink.state.benchmark.StateBenchmarkBase;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.infra.Blackhole;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+import org.openjdk.jmh.runner.options.VerboseMode;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static 
org.apache.flink.state.benchmark.StateBackendBenchmarkUtils.applyToAllKeys;
+import static 
org.apache.flink.state.benchmark.StateBackendBenchmarkUtils.compactState;
+import static 
org.apache.flink.state.benchmark.StateBackendBenchmarkUtils.getListState;
+import static 
org.apache.flink.state.benchmark.StateBenchmarkConstants.listValueCount;
+import static 
org.apache.flink.state.benchmark.StateBenchmarkConstants.setupKeyCount;
+
+/** Implementation for list state benchmark testing. */
+public class TtlListStateBenchmark extends TtlStateBenchmarkBase {
+    private final String STATE_NAME = "listState";
+    private final ListStateDescriptor<Long> STATE_DESC =
+            configTtl(new ListStateDescriptor<>(STATE_NAME, Long.class));
+    private ListState<Long> listState;
+    private List<Long> dummyLists;
+
+    public static void main(String[] args) throws RunnerException {
+        Options opt =
+                new OptionsBuilder()
+                        .verbosity(VerboseMode.NORMAL)
+                        .include(".*" + 
TtlListStateBenchmark.class.getCanonicalName() + ".*")
+                        .build();
+
+        new Runner(opt).run();
+    }
+
+    @Setup
+    public void setUp() throws Exception {
+        keyedStateBackend = createKeyedStateBackend();
+        listState = getListState(keyedStateBackend, STATE_DESC);
+        dummyLists = new ArrayList<>(listValueCount);
+        for (int i = 0; i < listValueCount; ++i) {
+            dummyLists.add(random.nextLong());
+        }
+        keyIndex = new AtomicInteger();
+    }
+
+    @Setup(Level.Iteration)
+    public void setUpPerIteration() throws Exception {
+        for (int i = 0; i < setupKeyCount; ++i) {
+            keyedStateBackend.setCurrentKey((long) i);
+            listState.add(random.nextLong());
+        }
+        // make sure only one sst file left, so all get invocation will access 
this single file,
+        // to prevent the spike caused by different key distribution in 
multiple sst files,
+        // the more access to the older sst file, the lower throughput will be.
+        if (keyedStateBackend instanceof RocksDBKeyedStateBackend) {

Review Comment:
   I noticed that there existed so much duplicated code here, can how reduce 
this code via Generics or more abstract class?



##########
src/main/java/org/apache/flink/state/benchmark/ttl/TtlStateBenchmarkBase.java:
##########
@@ -0,0 +1,48 @@
+package org.apache.flink.state.benchmark.ttl;
+
+import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.state.StateTtlConfig;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.state.benchmark.StateBenchmarkBase;
+import org.openjdk.jmh.annotations.Param;
+
+import java.util.concurrent.TimeUnit;
+
+/** The base class for state tests with ttl. */
+public class TtlStateBenchmarkBase extends StateBenchmarkBase {
+
+    /** The expired time of ttl. */
+    public enum ExpiredTimeOptions {
+
+        /** 5 seconds. */
+        Seconds5(5000),

Review Comment:
   What's the performance/result change if we tune this expired-seconds 
configuration?
   Say if we increase it to `10` seconds, or decrease it to `3` seconds?



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

Reply via email to