You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/01/11 16:23:26 UTC

[GitHub] [arrow] bkietz commented on a change in pull request #8210: ARROW-10031: [CI][Java] Support Java benchmark in Archery

bkietz commented on a change in pull request #8210:
URL: https://github.com/apache/arrow/pull/8210#discussion_r555119488



##########
File path: dev/archery/archery/benchmark/compare.py
##########
@@ -54,6 +54,14 @@ def formatter_for_unit(unit):
         return bytes_per_seconds_fmt
     elif unit == "items_per_second":
         return items_per_seconds_fmt
+    elif unit.startswith("ops/"):

Review comment:
       Instead, please convert to "items_per_second" or "bytes_per_second" in `JavaMicrobenchmarkHarnessObservation.unit`

##########
File path: dev/archery/archery/benchmark/jmh.py
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from itertools import filterfalse, groupby, tee
+import json
+import subprocess
+from tempfile import NamedTemporaryFile
+
+from .core import Benchmark
+from ..utils.maven import Maven
+
+
+def partition(pred, iterable):
+    # adapted from python's examples
+    t1, t2 = tee(iterable)
+    return list(filter(pred, t1)), list(filterfalse(pred, t2))
+
+
+class JavaMicrobenchmarkHarnessCommand(Maven):
+    """ Run a Java Micro Benchmark Harness
+
+    This assumes the binary supports the standard command line options,
+    notably `--benchmark_filter`, `--benchmark_format`, etc...
+    """
+
+    def __init__(self, build, benchmark_filter=None):
+        self.benchmark_filter = benchmark_filter
+        self.build = build
+        self.maven = Maven()
+
+    def list_benchmarks(self):
+        argv = []
+        if self.benchmark_filter:
+            argv.append("-Dbenchmark.filter={}".format(self.benchmark_filter))
+        result = self.build.list(
+            *argv, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
+
+        """ Extract benchmark names from output. Assume the following output

Review comment:
       Please put docstrings on the first line of function definitions

##########
File path: dev/archery/archery/benchmark/jmh.py
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from itertools import filterfalse, groupby, tee
+import json
+import subprocess
+from tempfile import NamedTemporaryFile
+
+from .core import Benchmark
+from ..utils.maven import Maven
+
+
+def partition(pred, iterable):
+    # adapted from python's examples
+    t1, t2 = tee(iterable)
+    return list(filter(pred, t1)), list(filterfalse(pred, t2))
+
+
+class JavaMicrobenchmarkHarnessCommand(Maven):
+    """ Run a Java Micro Benchmark Harness
+
+    This assumes the binary supports the standard command line options,
+    notably `--benchmark_filter`, `--benchmark_format`, etc...
+    """
+
+    def __init__(self, build, benchmark_filter=None):
+        self.benchmark_filter = benchmark_filter
+        self.build = build
+        self.maven = Maven()
+
+    def list_benchmarks(self):
+        argv = []
+        if self.benchmark_filter:
+            argv.append("-Dbenchmark.filter={}".format(self.benchmark_filter))
+        result = self.build.list(
+            *argv, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
+
+        """ Extract benchmark names from output. Assume the following output
+          ...
+          Benchmarks:
+          org.apache.arrow.vector.IntBenchmarks.setIntDirectly
+          ...
+          org.apache.arrow.vector.IntBenchmarks.setWithValueHolder
+          org.apache.arrow.vector.IntBenchmarks.setWithWriter
+          ...
+          [INFO]
+        """
+
+        lists = []
+        benchmarks = False
+        for line in str.splitlines(result.stdout.decode("utf-8")):
+            if not benchmarks:
+                if line.startswith("Benchmarks:"):
+                    benchmarks = True
+            else:
+                if line.startswith("org.apache.arrow"):
+                    lists.append(line)
+                if line.startswith("[INFO]"):
+                    benchmarks = False

Review comment:
       ```suggestion
   ```

##########
File path: dev/archery/archery/cli.py
##########
@@ -367,11 +381,21 @@ def benchmark_common_options(cmd):
         click.option("--output", metavar="<output>",
                      type=click.File("w", encoding="utf8"), default="-",
                      help="Capture output result into file."),
+        click.option("--language", metavar="<lang>", type=str, default="cpp",
+                     show_default=True, callback=check_language,
+                     help="Specify target language for the benchmark"),
+        click.option("--build-extras", type=str, multiple=True,
+                     help="Extra flags/options to pass to mvn build. "
+                     "Can be stackd. For language=java"),

Review comment:
       ```suggestion
                        "Can be stacked. For language=java"),
   ```

##########
File path: dev/archery/archery/cli.py
##########
@@ -394,16 +418,53 @@ def benchmark_filter_options(cmd):
 @benchmark_common_options
 @click.pass_context
 def benchmark_list(ctx, rev_or_path, src, preserve, output, cmake_extras,
-                   **kwargs):
+                   language, build_extras, benchmark_extras, java_home,
+                   java_options, **kwargs):
     """ List benchmark suite.
     """
+    if language == "cpp":
+        ctx.forward(benchmark_list_cpp, **kwargs)
+    elif language == "java":
+        ctx.forward(benchmark_list_java, **kwargs)
+
+
+@benchmark.command(name="list_cpp")
+@click.pass_context
+def benchmark_list_cpp(ctx, rev_or_path, src, preserve, output, cmake_extras,

Review comment:
       This division seems unnecessary. Please just keep a single `benchmark_list` function (moving the branch on language into the `with` block)

##########
File path: dev/archery/archery/lang/java.py
##########
@@ -28,3 +31,48 @@ def __init__(self, jar, *args, **kwargs):
         self.jar = jar
         self.argv = ("-jar", jar)
         Java.__init__(self, *args, **kwargs)
+
+
+class JavaConfiguration:
+    def __init__(self,
+
+                 # toolchain
+                 java_home=None, java_options=None,
+                 # build & benchmark
+                 build_extras=None, benchmark_extras=None):
+        self.java_home = java_home
+        self.java_options = java_options
+
+        self.build_extras = build_extras
+        self.benchmark_extras = benchmark_extras
+
+    @property
+    def build_definitions(self):
+        extras = list(self.build_extras) if self.build_extras else []
+        return extras

Review comment:
       I'd recommend doing the cast-to-list here; if an iterator were passed for `build_extras` then it'd be depleted after the first access to `build_definitions`:
   ```suggestion
           self.build_extras = list(build_extras) if build_extras else []
           self.benchmark_extras = list(benchmark_extras) if benchmark_extras else []
   
       @property
       def build_definitions(self):
           return self.build_extras
   ```
   
   I see that CppConfiguration has this same flaw; it's not necessary for you to address it there too

##########
File path: dev/archery/archery/benchmark/jmh.py
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from itertools import filterfalse, groupby, tee
+import json
+import subprocess
+from tempfile import NamedTemporaryFile
+
+from .core import Benchmark
+from ..utils.maven import Maven
+
+
+def partition(pred, iterable):
+    # adapted from python's examples
+    t1, t2 = tee(iterable)
+    return list(filter(pred, t1)), list(filterfalse(pred, t2))
+
+
+class JavaMicrobenchmarkHarnessCommand(Maven):
+    """ Run a Java Micro Benchmark Harness
+
+    This assumes the binary supports the standard command line options,
+    notably `--benchmark_filter`, `--benchmark_format`, etc...
+    """
+
+    def __init__(self, build, benchmark_filter=None):
+        self.benchmark_filter = benchmark_filter
+        self.build = build
+        self.maven = Maven()
+
+    def list_benchmarks(self):
+        argv = []
+        if self.benchmark_filter:
+            argv.append("-Dbenchmark.filter={}".format(self.benchmark_filter))
+        result = self.build.list(
+            *argv, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
+
+        """ Extract benchmark names from output. Assume the following output

Review comment:
       Also, please make it clear that you're using the strings "Benchmarks:" and "[INFO]" to delimit lines containing benchmark names

##########
File path: dev/archery/archery/benchmark/jmh.py
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from itertools import filterfalse, groupby, tee
+import json
+import subprocess
+from tempfile import NamedTemporaryFile
+
+from .core import Benchmark
+from ..utils.maven import Maven
+
+
+def partition(pred, iterable):
+    # adapted from python's examples
+    t1, t2 = tee(iterable)
+    return list(filter(pred, t1)), list(filterfalse(pred, t2))
+
+
+class JavaMicrobenchmarkHarnessCommand(Maven):

Review comment:
       It's odd that this class inherits `Maven` instead of `Command`, especially since it has a member `self.maven`
   ```suggestion
   class JavaMicrobenchmarkHarnessCommand(Command):
   ```

##########
File path: dev/archery/archery/cli.py
##########
@@ -357,6 +366,11 @@ def benchmark(ctx):
 
 
 def benchmark_common_options(cmd):
+    def check_language(ctx, param, value):
+        if value != "cpp" and value != "java":

Review comment:
       ```suggestion
           if value not in {"cpp", "java"}:
   ```

##########
File path: dev/archery/archery/benchmark/jmh.py
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from itertools import filterfalse, groupby, tee
+import json
+import subprocess
+from tempfile import NamedTemporaryFile
+
+from .core import Benchmark
+from ..utils.maven import Maven
+
+
+def partition(pred, iterable):
+    # adapted from python's examples
+    t1, t2 = tee(iterable)
+    return list(filter(pred, t1)), list(filterfalse(pred, t2))
+
+
+class JavaMicrobenchmarkHarnessCommand(Maven):
+    """ Run a Java Micro Benchmark Harness
+
+    This assumes the binary supports the standard command line options,
+    notably `--benchmark_filter`, `--benchmark_format`, etc...

Review comment:
       This comment doesn't apply to java benchmarks; seems copy pasted from GoogleBenchmarkCommand?

##########
File path: dev/archery/archery/benchmark/jmh.py
##########
@@ -0,0 +1,165 @@
+# 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.
+
+from itertools import filterfalse, groupby, tee
+import json
+import subprocess
+from tempfile import NamedTemporaryFile
+
+from .core import Benchmark
+from ..utils.maven import Maven
+
+
+def partition(pred, iterable):
+    # adapted from python's examples
+    t1, t2 = tee(iterable)
+    return list(filter(pred, t1)), list(filterfalse(pred, t2))
+
+
+class JavaMicrobenchmarkHarnessCommand(Maven):
+    """ Run a Java Micro Benchmark Harness
+
+    This assumes the binary supports the standard command line options,
+    notably `--benchmark_filter`, `--benchmark_format`, etc...
+    """
+
+    def __init__(self, build, benchmark_filter=None):
+        self.benchmark_filter = benchmark_filter
+        self.build = build
+        self.maven = Maven()
+
+    def list_benchmarks(self):
+        argv = []
+        if self.benchmark_filter:
+            argv.append("-Dbenchmark.filter={}".format(self.benchmark_filter))
+        result = self.build.list(
+            *argv, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
+
+        """ Extract benchmark names from output. Assume the following output
+          ...
+          Benchmarks:
+          org.apache.arrow.vector.IntBenchmarks.setIntDirectly
+          ...
+          org.apache.arrow.vector.IntBenchmarks.setWithValueHolder
+          org.apache.arrow.vector.IntBenchmarks.setWithWriter
+          ...
+          [INFO]
+        """
+
+        lists = []
+        benchmarks = False
+        for line in str.splitlines(result.stdout.decode("utf-8")):
+            if not benchmarks:
+                if line.startswith("Benchmarks:"):
+                    benchmarks = True
+            else:
+                if line.startswith("org.apache.arrow"):
+                    lists.append(line)
+                if line.startswith("[INFO]"):
+                    benchmarks = False
+                    break
+        return lists
+
+    def results(self, repetitions=1):
+        with NamedTemporaryFile(suffix=".json") as out:
+            argv = ["-Dbenchmark.runs={}".format(repetitions),
+                    "-Dbenchmark.resultfile={}".format(out.name),
+                    "-Dbenchmark.resultformat=json"]
+            if self.benchmark_filter:
+                argv.append(
+                    "-Dbenchmark.filter={}".format(self.benchmark_filter)
+                )
+
+            self.build.benchmark(*argv, check=True)
+            return json.load(out)
+
+
+class JavaMicrobenchmarkHarnessObservation:
+    """ Represents one run of a single Java Microbenchmark Harness
+    """
+
+    def __init__(self, benchmark, primaryMetric,
+                 forks, warmupIterations, measurementIterations, **counters):
+        self.name = benchmark
+        self.primaryMetric = primaryMetric
+        self.score = primaryMetric["score"]
+        self.scoreUnit = primaryMetric["scoreUnit"]
+        self.forks = forks
+        self.warmups = warmupIterations
+        self.runs = measurementIterations
+        self.counters = {
+            "mode": counters["mode"],
+            "threads": counters["threads"],
+            "warmups": warmupIterations,
+            "warmupTime": counters["warmupTime"],
+            "measurements": measurementIterations,
+            "measurementTime": counters["measurementTime"],
+            "jvmArgs": counters["jvmArgs"]
+        }
+
+    @property
+    def value(self):
+        """ Return the benchmark value."""
+        return self.score
+
+    @property
+    def unit(self):
+        return self.scoreUnit
+
+    def __repr__(self):
+        return str(self.value)
+
+
+class JavaMicrobenchmarkHarness(Benchmark):
+    """ A set of JavaMicrobenchmarkHarnessObservations. """
+
+    def __init__(self, name, runs):
+        """ Initialize a JavaMicrobenchmarkHarness.
+
+        Parameters
+        ----------
+        name: str
+              Name of the benchmark
+        forks: int
+        warmups: int
+        runs: int
+        runs: list(JavaMicrobenchmarkHarnessObservation)
+              Repetitions of JavaMicrobenchmarkHarnessObservation run.
+
+        """
+        self.name = name
+        self.runs = sorted(runs, key=lambda b: b.value)
+        unit = self.runs[0].unit
+        less_is_better = unit.endswith("/op")
+        values = [b.value for b in self.runs]
+        # Slight kludge to extract the UserCounters for each benchmark
+        self.counters = self.runs[0].counters
+        super().__init__(name, unit, less_is_better, values)
+
+    def __repr__(self):
+        return "JavaMicrobenchmarkHarness[name={},runs={}]".format(

Review comment:
       Nit: why include the string "Harness" in all these class names? Seems unnecessary.

##########
File path: dev/archery/archery/cli.py
##########
@@ -450,19 +511,57 @@ def benchmark_run(ctx, rev_or_path, src, preserve, output, cmake_extras,
     \b
     archery benchmark run --output=run.json
     """
+    if language == "cpp":
+        ctx.forward(benchmark_run_cpp, **kwargs)
+    elif language == "java":
+        ctx.forward(benchmark_run_java, **kwargs)
+
+
+@benchmark.command(name="run_cpp")
+@click.pass_context
+def benchmark_run_cpp(ctx, rev_or_path, src, preserve, output, cmake_extras,

Review comment:
       This division also seems unnecessary; CppBenchmarkRunner and JavaBenchmarkRunner seem similar enough to share more code here

##########
File path: dev/archery/archery/cli.py
##########
@@ -367,11 +381,21 @@ def benchmark_common_options(cmd):
         click.option("--output", metavar="<output>",
                      type=click.File("w", encoding="utf8"), default="-",
                      help="Capture output result into file."),
+        click.option("--language", metavar="<lang>", type=str, default="cpp",
+                     show_default=True, callback=check_language,
+                     help="Specify target language for the benchmark"),
+        click.option("--build-extras", type=str, multiple=True,
+                     help="Extra flags/options to pass to mvn build. "
+                     "Can be stackd. For language=java"),
+        click.option("--benchmark-extras", type=str, multiple=True,
+                     help="Extra flags/options to pass to mvn benchmark. "
+                     "Can be stackd. For language=java"),

Review comment:
       ```suggestion
                        "Can be stacked. For language=java"),
   ```

##########
File path: dev/archery/archery/cli.py
##########
@@ -550,19 +649,34 @@ def benchmark_diff(ctx, src, preserve, output, cmake_extras,
     # This should not recompute the benchmark from run.json
     archery --quiet benchmark diff WORKSPACE run.json > result.json
     """
+    if language == "cpp":
+        ctx.forward(benchmark_diff_cpp, **kwargs)
+    elif language == "java":
+        ctx.forward(benchmark_diff_java, **kwargs)
+
+
+@benchmark.command(name="diff_cpp")
+@click.pass_context
+def benchmark_diff_cpp(ctx, src, preserve, output, language, cmake_extras,

Review comment:
       Again, why separate the languages' benchmarks? It seems especially valuable to have a clearly unified strategy for comparison of benchmarks

##########
File path: dev/archery/archery/benchmark/runner.py
##########
@@ -205,3 +176,133 @@ def suites(self):
                 continue
 
             yield suite
+
+    @staticmethod
+    def from_rev_or_path(src, root, rev_or_path, cmake_conf, **kwargs):
+        """ Returns a BenchmarkRunner from a path or a git revision.
+
+        First, it checks if `rev_or_path` is a valid path (or string) of a json
+        object that can deserialize to a BenchmarkRunner. If so, it initialize
+        a StaticBenchmarkRunner from it. This allows memoizing the result of a
+        run in a file or a string.
+
+        Second, it checks if `rev_or_path` points to a valid CMake build
+        directory.  If so, it creates a CppBenchmarkRunner with this existing
+        CMakeBuild.
+
+        Otherwise, it assumes `rev_or_path` is a revision and clone/checkout
+        the given revision and create a fresh CMakeBuild.
+        """
+        build = None
+        if StaticBenchmarkRunner.is_json_result(rev_or_path):
+            return StaticBenchmarkRunner.from_json(rev_or_path, **kwargs)
+        elif CMakeBuild.is_build_dir(rev_or_path):
+            build = CMakeBuild.from_path(rev_or_path)
+            return CppBenchmarkRunner(build, **kwargs)
+        else:
+            # Revisions can references remote via the `/` character, ensure
+            # that the revision is path friendly
+            path_rev = rev_or_path.replace("/", "_")
+            root_rev = os.path.join(root, path_rev)
+            os.mkdir(root_rev)
+
+            clone_dir = os.path.join(root_rev, "arrow")
+            # Possibly checkout the sources at given revision, no need to
+            # perform cleanup on cloned repository as root_rev is reclaimed.
+            src_rev, _ = src.at_revision(rev_or_path, clone_dir)
+            cmake_def = CppCMakeDefinition(src_rev.cpp, cmake_conf)
+            build_dir = os.path.join(root_rev, "build")
+            return CppBenchmarkRunner(cmake_def.build(build_dir), **kwargs)
+
+
+class JavaBenchmarkRunner(BenchmarkRunner):
+    """ Run suites for Java. """
+
+    def __init__(self, build, **kwargs):
+        """ Initialize a JavaBenchmarkRunner. """
+        self.build = build
+        self.repetitions = 5  # default repetitions for Java
+        super().__init__(**kwargs)
+
+    @staticmethod
+    def default_configuration(**kwargs):
+        """ Returns the default benchmark configuration. """
+        return JavaConfiguration(**kwargs)
+
+    def suite(self, name):
+        """ Returns the resulting benchmarks for a given suite. """
+        # update .m2 directory, which installs target jars
+        self.build.build()
+
+        suite_cmd = JavaMicrobenchmarkHarnessCommand(
+            self.build, self.benchmark_filter)
+
+        # Ensure there will be data
+        benchmark_names = suite_cmd.list_benchmarks()
+        if not benchmark_names:
+            return None
+
+        results = suite_cmd.results(repetitions=self.repetitions)
+        benchmarks = JavaMicrobenchmarkHarness.from_json(results)
+        return BenchmarkSuite(name, benchmarks)
+
+    @property
+    def list_benchmarks(self):
+        """ Returns all suite names """
+        # Ensure build is up-to-date to run benchmarks
+        self.build.build()
+
+        suite_cmd = JavaMicrobenchmarkHarnessCommand(self.build)
+        benchmark_names = suite_cmd.list_benchmarks()
+        for benchmark_name in benchmark_names:
+            yield "{}".format(benchmark_name)
+
+    @property
+    def suites(self):
+        """ Returns all suite for a runner. """
+        suite_name = "JavaBenchmark"
+        suite = self.suite(suite_name)
+
+        # Filter may exclude all benchmarks
+        if not suite:
+            logger.debug("Suite {} executed but no results"
+                         .format(suite_name))
+            return
+
+        yield suite
+
+    @staticmethod
+    def from_rev_or_path(src, root, rev_or_path, maven_conf, **kwargs):
+        """ Returns a BenchmarkRunner from a path or a git revision.
+
+        First, it checks if `rev_or_path` is a valid path (or string) of a json
+        object that can deserialize to a BenchmarkRunner. If so, it initialize
+        a StaticBenchmarkRunner from it. This allows memoizing the result of a
+        run in a file or a string.
+
+        Second, it checks if `rev_or_path` points to a valid Maven build
+        directory.  If so, it creates a JavaenchmarkRunner with this existing

Review comment:
       ```suggestion
           directory.  If so, it creates a JavaBenchmarkRunner with this existing
   ```
   Again, seems like this code didn't need to be repeated




----------------------------------------------------------------
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:
users@infra.apache.org