You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/07/07 14:44:18 UTC

[GitHub] [beam] mosche opened a new pull request, #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

mosche opened a new pull request, #22182:
URL: https://github.com/apache/beam/pull/22182

   This adds a new `jmh` module for Java `core` with benchmarks of `GetterBasedSchemaProvider` as suggested in https://github.com/apache/beam/pull/17203.
   
   ![Screenshot 2022-07-07 at 16 42 13](https://user-images.githubusercontent.com/1401430/177801591-46110ff6-9338-4af8-9a97-c74245b21ca4.png)
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Mention the appropriate issue in your description (for example: `addresses #123`), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment `fixes #<ISSUE NUMBER>` instead.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r920454143


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   My idea was to have coverage to be able to get a field from a row (aka read) and set a field on a row (aka write). The use cases I was thinking for a given row would be that a user would get the same field from the same row once or multiple times and similarly would set a field on the same row once or multiple times. As you have done I would repeat this for many rows.
   
   This would highlight the one time cost of getting/setting a field and also show the cost of getting/setting a field multiple times.



##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/GetterBasedSchemaProviderBenchmark.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static org.apache.beam.sdk.schemas.RowBundles.ByteBufferBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.BytesBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.DateTimeBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.IntBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.MapOfIntBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.MapOfNestedIntBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.NestedBytesBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.NestedIntBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.StringBuilderBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.StringBundle;
+
+import org.apache.beam.sdk.schemas.RowBundles.ArrayOfNestedStringBundle;
+import org.apache.beam.sdk.schemas.RowBundles.ArrayOfStringBundle;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Benchmarks for {@link GetterBasedSchemaProvider} on reading / writing fields based on {@link
+ * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor) toRowFunction} / {@link
+ * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor) fromRowFunction}.
+ *
+ * <p>Each benchmark method invocation, depending on {@link RowBundle#action}, either reads a bundle

Review Comment:
   the comment needs updating



##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import java.nio.charset.StandardCharsets;
+import java.util.function.Function;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_REPEATED(Factory::createWithGetter);
+
+    final Function<SchemaCoder<?>, Factory<Row>> factoryProvider;
+
+    Action(Function<SchemaCoder<?>, Factory<Row>> factoryProvider) {
+      this.factoryProvider = factoryProvider;
+    }
+  }
+
+  private static final SchemaRegistry REGISTRY = SchemaRegistry.createDefault();
+
+  private final SchemaCoder<?> coder;
+  private final SerializableFunction<Row, ?> fromRow;
+  private Factory<Row> factory;
+
+  protected Row[] rows;
+
+  @Param("100000")
+  int bundleSize;
+
+  @Param({"READ_REPEATED", "WRITE"})

Review Comment:
   ```suggestion
     @Param({"READ_ONCE", "READ_REPEATED", "WRITE"})
   ```



##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/package-info.java:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+/** Benchmarks for the Java SDK schemas. */

Review Comment:
   ```suggestion
   /** Benchmarks for schemas. */
   ```



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on PR #22182:
URL: https://github.com/apache/beam/pull/22182#issuecomment-1180436296

   Thanks a million for your thorough review and profiling the code, @lukecwik πŸ’― 
   
   > I'm worried that the timed portion is being dominated by the benchmark code and not the code that we want to measure.
   
   I do agree that this is a huge concern and it's been tricky in this case. I've moved as much of the overhead into `setup` to prepare an array of rows for each benchmark iteration. I've optimised `setup` code according to your suggestion (using `for` loops), but the overhead of iterating over that array will remain (the same) during benchmark.
   
   In case of the simple int field that was really significant, thanks for catching that. It's interesting to see how costly nested structures are (e.g. processNestedIntField) in comparison, in that case the effect was much less.
   
   Switching to `READ_REPEATED` (as you suggested) reduced the proportional overhead and removing the nested loop in [readField](https://github.com/apache/beam/pull/22182/commits/95c425111d0c0fb5948fe46bbb9ad071b138f355#diff-d1eec5d84b10a3d22d612c37df41ec483b3058034c8436932e77ac808690ed76L134-R149) (replaced by `readRowsOnce`, `readRowsRepeatedly`) also helped a bit. But in the end the code doesn't do much more than throwing the row into a sink. 
   
   For collection types, I've reduced the size to one in the factory to avoid having to iterate over the collections when reading. Nevertheless, given there might be collections of rows (with getters itself), I think it's important to access these rather than throwing the entire collection into the blackhole.
   
   If you see any possible further optimisations, I'd be glad to hear. 
   To make things easier to follow when profiling, I've replaced the lambdas with classes.
   
   Read `int` field once:
   <img width="829" alt="Screenshot 2022-07-11 at 14 19 06" src="https://user-images.githubusercontent.com/1401430/178271981-baab50c5-2b9a-4220-8e98-159d4af4837a.png">
   Read `int` field repeatedly:
   <img width="828" alt="Screenshot 2022-07-11 at 14 19 20" src="https://user-images.githubusercontent.com/1401430/178271969-84d4ce9e-9d25-48ad-82d6-90e21dbacc88.png">
   Read nested `int` field repeatedly:
   <img width="829" alt="Screenshot 2022-07-11 at 14 19 32" src="https://user-images.githubusercontent.com/1401430/178271957-e47b054e-970a-4cfa-a41a-b106c37b7de8.png">
   
   


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r921812824


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/GetterBasedSchemaProviderBenchmark.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static org.apache.beam.sdk.schemas.RowBundles.ByteBufferBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.BytesBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.DateTimeBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.IntBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.MapOfIntBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.MapOfNestedIntBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.NestedBytesBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.NestedIntBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.StringBuilderBundle;
+import static org.apache.beam.sdk.schemas.RowBundles.StringBundle;
+
+import org.apache.beam.sdk.schemas.RowBundles.ArrayOfNestedStringBundle;
+import org.apache.beam.sdk.schemas.RowBundles.ArrayOfStringBundle;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Benchmarks for {@link GetterBasedSchemaProvider} on reading / writing fields based on {@link
+ * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor) toRowFunction} / {@link
+ * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor) fromRowFunction}.
+ *
+ * <p>Each benchmark method invocation, depending on {@link RowBundle#action}, either reads a bundle

Review Comment:
   πŸ‘ thx



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r917895159


##########
sdks/java/core/jmh/build.gradle:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+
+applyJavaNature(
+  automaticModuleName: 'org.apache.beam.sdk.jmh',
+  enableJmh: true,
+  publish: false)
+
+description = "Apache Beam :: SDKs :: Java :: Core :: JMH"
+ext.summary = "This contains JMH benchmarks for the SDK Core for Beam Java"
+
+configurations {
+  jammAgent
+}
+
+dependencies {
+  implementation project(path: ":sdks:java:core", configuration: "shadow")
+  implementation library.java.joda_time
+  runtimeOnly library.java.slf4j_jdk14
+  jammAgent library.java.jamm
+}
+
+jmh {
+  configurations.jammAgent.resolvedConfiguration.files.each {
+    jvmArgs '-javaagent:' + it
+  }
+}
+
+jmhTest {
+  configurations.jammAgent.resolvedConfiguration.files.each {
+    jvmArgs '-javaagent:' + it
+  }
+}
+

Review Comment:
   πŸ‘ removed



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r917001567


##########
sdks/java/core/jmh/build.gradle:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+
+applyJavaNature(
+  automaticModuleName: 'org.apache.beam.sdk.jmh',
+  enableJmh: true,
+  publish: false)
+
+description = "Apache Beam :: SDKs :: Java :: Core :: JMH"
+ext.summary = "This contains JMH benchmarks for the SDK Core for Beam Java"
+
+configurations {
+  jammAgent
+}
+
+dependencies {
+  implementation project(path: ":sdks:java:core", configuration: "shadow")
+  implementation library.java.joda_time
+  runtimeOnly library.java.slf4j_jdk14
+  jammAgent library.java.jamm

Review Comment:
   ```suggestion
   ```



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r917907776


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_REPEATED(Factory::createWithGetter);
+
+    final Function<SchemaCoder<?>, Factory<Row>> factoryProvider;
+
+    Action(Function<SchemaCoder<?>, Factory<Row>> factoryProvider) {
+      this.factoryProvider = factoryProvider;
+    }
+  }
+
+  private static final SchemaRegistry REGISTRY = SchemaRegistry.createDefault();
+
+  private final SchemaCoder<?> coder;
+  private final Sink<Row> sink;
+  private Factory<Row> factory;
+
+  protected Row[] rows;
+
+  @Param("100000")
+  int bundleSize;
+
+  @Param({"READ_ONCE", "WRITE"})

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

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r924774823


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   My idea was to have coverage to be able to get a field from a row (aka read) and set a field on a row (aka write). The use cases I was thinking for a given row would be that a user would get the same field from the same row once or multiple times and similarly would set a field on the same row once or multiple times. But as you have mentioned that doesn't make sense.
   
   Lets merge as is and if you think of something or if I think of something we can bring it up as a new PR or a discussion on the dev@ list.



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r924494192


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   > set a field on the same row once or multiple times
   
   @lukecwik Looks like the above is the only open point remaining.
   Any thoughts on my answer? I might simply be missing something here.
   
   > It doesn't look like this is supported in the context of the GetterBasedSchemaProvider, neither RowWithGetters nor RowWithStorage support mutating / setting field values. Setters are used (exclusively?) when invoking the fromRowFunction to set field values on a new empty instance (or create that instance via some creator).



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r918242777


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   If that is the case I would benchmark the cost of the first read, 1000 repeated reads, first write, 1000 repeated writes (this one is the least likely to happen but it is nice to have for completeness).
   
   This covers the common scenarios since most fields are typically read/written once followed by the case where they are read a lot and then followed by the case where they are written a lot



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r922330620


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   If we have these running regularly we can reduce the amount of forks from 5.



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r921852047


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   Ok, that makes sense two me, thanks for the clarification. I haven't looked too deeply into the available sql transforms on rows. It doesn't look like this is supported in the context of the `GetterBasedSchemaProvider`, neither `RowWithGetters` nor `RowWithStorage` support mutating / setting field values. Setters are used (exclusively?) when invoking the `fromRowFunction` to set field values on a new empty instance (or create that instance via some creator). 
   
   Another thing to consider when adding more benchmarks is probably the total runtime. With the current default configuration a single benchmark takes around 500secs (warmups, iterations, forks) + some overhead. In total that already sums up to ~ 5h (12 runs per each of the 3 actions).
   
   



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] codecov[bot] commented on pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #22182:
URL: https://github.com/apache/beam/pull/22182#issuecomment-1177778769

   # [Codecov](https://codecov.io/gh/apache/beam/pull/22182?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#22182](https://codecov.io/gh/apache/beam/pull/22182?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2655c1a) into [master](https://codecov.io/gh/apache/beam/commit/df162c1e2fb221c64cd861605fb35b37d2e6b8ec?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (df162c1) will **decrease** coverage by `0.02%`.
   > The diff coverage is `n/a`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #22182      +/-   ##
   ==========================================
   - Coverage   74.23%   74.21%   -0.03%     
   ==========================================
     Files         702      702              
     Lines       92873    92873              
   ==========================================
   - Hits        68945    68926      -19     
   - Misses      22661    22680      +19     
     Partials     1267     1267              
   ```
   
   | Flag | Coverage Ξ” | |
   |---|---|---|
   | python | `83.59% <ΓΈ> (-0.03%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/22182?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Ξ” | |
   |---|---|---|
   | [...ks/python/apache\_beam/runners/worker/data\_plane.py](https://codecov.io/gh/apache/beam/pull/22182/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvZGF0YV9wbGFuZS5weQ==) | `87.57% <0.00%> (-1.70%)` | :arrow_down: |
   | [sdks/python/apache\_beam/io/source\_test\_utils.py](https://codecov.io/gh/apache/beam/pull/22182/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vc291cmNlX3Rlc3RfdXRpbHMucHk=) | `88.01% <0.00%> (-1.39%)` | :arrow_down: |
   | [...eam/runners/portability/fn\_api\_runner/execution.py](https://codecov.io/gh/apache/beam/pull/22182/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9mbl9hcGlfcnVubmVyL2V4ZWN1dGlvbi5weQ==) | `92.44% <0.00%> (-0.65%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/22182/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...eam/runners/interactive/interactive\_environment.py](https://codecov.io/gh/apache/beam/pull/22182/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9lbnZpcm9ubWVudC5weQ==) | `91.71% <0.00%> (-0.31%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/22182/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `88.94% <0.00%> (-0.16%)` | :arrow_down: |
   | [sdks/python/apache\_beam/transforms/util.py](https://codecov.io/gh/apache/beam/pull/22182/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy91dGlsLnB5) | `96.06% <0.00%> (-0.16%)` | :arrow_down: |
   | [...hon/apache\_beam/runners/worker/bundle\_processor.py](https://codecov.io/gh/apache/beam/pull/22182/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvYnVuZGxlX3Byb2Nlc3Nvci5weQ==) | `93.54% <0.00%> (-0.13%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/22182?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Ξ” = absolute <relative> (impact)`, `ΓΈ = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/22182?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [df162c1...2655c1a](https://codecov.io/gh/apache/beam/pull/22182?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r918939204


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   I'm not sure I fully understand what you are proposing @lukecwik ...
   
   It might be that the terms `read` / `write` are not entirely correct & symmetric in this case:
   - `read`: invoke getter on the underlying object via `RowWithGetter`(for a single field in this benchmark)
   - `write`: create new instance of the type represented by `RowWithStorage` and invoke the setter to write the field (again for a single field in this benchmark) (`RowWithGetter` would simply return the underlying object, that's the trivial, cheap path)
   
   I think it's valuable to (also) have separate benchmarks for both these to help isolate problems in case of any performance degradation. But having a more use case driven benchmark would certainly be great. Could you elaborate a bit more on the above? 



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r917894908


##########
sdks/java/core/jmh/build.gradle:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+
+applyJavaNature(
+  automaticModuleName: 'org.apache.beam.sdk.jmh',
+  enableJmh: true,
+  publish: false)
+
+description = "Apache Beam :: SDKs :: Java :: Core :: JMH"
+ext.summary = "This contains JMH benchmarks for the SDK Core for Beam Java"
+
+configurations {
+  jammAgent
+}
+

Review Comment:
   πŸ‘ removed



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r917897629


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   To be honest 3 is just a blind guess... the intention of repeated reads is to reflect the behavior of the cache in RowWithGetters 



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik commented on PR #22182:
URL: https://github.com/apache/beam/pull/22182#issuecomment-1177901814

   > @lukecwik @TheNeuralBit @apilloud Finally found time to follow up on #17203.
   > 
   > > It could make sense to run these benchmarks continuously and upload metrics to s.apache.org/beam-community-metrics
   > 
   > I had a brief look at s.apache.org/beam-community-metrics. So far there's no JMH metrics uploaded at all, right?
   
   Thats right, it had been brought up to start adding the jmh benchmarks to them so we can see a performance trend over time.


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on PR #22182:
URL: https://github.com/apache/beam/pull/22182#issuecomment-1178618460

   Run Java PreCommit


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on PR #22182:
URL: https://github.com/apache/beam/pull/22182#issuecomment-1177736477

   @lukecwik @TheNeuralBit @apilloud Finally found time to follow up on https://github.com/apache/beam/pull/17203.
   
   > It could make sense to run these benchmarks continuously and upload metrics to s.apache.org/beam-community-metrics
   
   I had a brief look at s.apache.org/beam-community-metrics. So far there's no JMH metrics uploaded at all, right?


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r916947148


##########
sdks/java/core/jmh/build.gradle:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+
+applyJavaNature(
+  automaticModuleName: 'org.apache.beam.sdk.jmh',
+  enableJmh: true,
+  publish: false)
+
+description = "Apache Beam :: SDKs :: Java :: Core :: JMH"
+ext.summary = "This contains JMH benchmarks for the SDK Core for Beam Java"
+
+configurations {
+  jammAgent
+}
+
+dependencies {
+  implementation project(path: ":sdks:java:core", configuration: "shadow")
+  implementation library.java.joda_time
+  runtimeOnly library.java.slf4j_jdk14
+  jammAgent library.java.jamm
+}
+
+jmh {
+  configurations.jammAgent.resolvedConfiguration.files.each {
+    jvmArgs '-javaagent:' + it
+  }
+}
+
+jmhTest {
+  configurations.jammAgent.resolvedConfiguration.files.each {
+    jvmArgs '-javaagent:' + it
+  }
+}
+

Review Comment:
   ```suggestion
   ```
   
   I don't think you need to add jamm as a java agent. It is only used in the SDK harness for measuring object sizes.



##########
sdks/java/core/jmh/build.gradle:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+
+applyJavaNature(
+  automaticModuleName: 'org.apache.beam.sdk.jmh',
+  enableJmh: true,
+  publish: false)
+
+description = "Apache Beam :: SDKs :: Java :: Core :: JMH"
+ext.summary = "This contains JMH benchmarks for the SDK Core for Beam Java"
+
+configurations {
+  jammAgent
+}
+

Review Comment:
   ```suggestion
   ```
   
   I don't think you need to add jamm as a java agent. It is only used in the SDK harness for measuring object sizes.



##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_REPEATED(Factory::createWithGetter);
+
+    final Function<SchemaCoder<?>, Factory<Row>> factoryProvider;
+
+    Action(Function<SchemaCoder<?>, Factory<Row>> factoryProvider) {
+      this.factoryProvider = factoryProvider;
+    }
+  }
+
+  private static final SchemaRegistry REGISTRY = SchemaRegistry.createDefault();
+
+  private final SchemaCoder<?> coder;
+  private final Sink<Row> sink;
+  private Factory<Row> factory;
+
+  protected Row[] rows;
+
+  @Param("100000")
+  int bundleSize;
+
+  @Param({"READ_ONCE", "WRITE"})
+  Action action;
+
+  public RowBundle() {
+    this(null); // unused, just to prevent warnings
+  }

Review Comment:
   Why do you need this constructor?



##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_REPEATED(Factory::createWithGetter);
+
+    final Function<SchemaCoder<?>, Factory<Row>> factoryProvider;
+
+    Action(Function<SchemaCoder<?>, Factory<Row>> factoryProvider) {
+      this.factoryProvider = factoryProvider;
+    }
+  }
+
+  private static final SchemaRegistry REGISTRY = SchemaRegistry.createDefault();
+
+  private final SchemaCoder<?> coder;
+  private final Sink<Row> sink;
+  private Factory<Row> factory;
+
+  protected Row[] rows;
+
+  @Param("100000")
+  int bundleSize;
+
+  @Param({"READ_ONCE", "WRITE"})

Review Comment:
   READ_REPEATED?



##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link

Review Comment:
   Why 3x?
   
   Is this the amount of reads seen in practice when a field is read multiple times?



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on PR #22182:
URL: https://github.com/apache/beam/pull/22182#issuecomment-1181703588

   I already tested / profiled the code with `for (int i =...)`, that didn't make any difference.
   
   > A larger improvement would come from getting rid of the sink/switch statements and writing the code directly to read/write the fields directly 
   
   The switch statements were only used once during setup to stack functions together to read the final value according to the schema. Though it still reduced the overhead a bit replacing that with explicit code πŸ‘ 
   
   Explicit `readField`:
   <img width="805" alt="Screenshot 2022-07-12 at 14 15 17" src="https://user-images.githubusercontent.com/1401430/178489843-e3e9d49e-0b64-4cfc-b7d1-fb37b22d08b4.png">
   
   Previous generated sink:
   <img width="800" alt="Screenshot 2022-07-12 at 14 15 54" src="https://user-images.githubusercontent.com/1401430/178489871-47a41e51-c64b-4f9d-a53a-05751d126d8b.png">
   
   


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r917907554


##########
sdks/java/core/jmh/build.gradle:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+plugins { id 'org.apache.beam.module' }
+
+applyJavaNature(
+  automaticModuleName: 'org.apache.beam.sdk.jmh',
+  enableJmh: true,
+  publish: false)
+
+description = "Apache Beam :: SDKs :: Java :: Core :: JMH"
+ext.summary = "This contains JMH benchmarks for the SDK Core for Beam Java"
+
+configurations {
+  jammAgent
+}
+
+dependencies {
+  implementation project(path: ":sdks:java:core", configuration: "shadow")
+  implementation library.java.joda_time
+  runtimeOnly library.java.slf4j_jdk14
+  jammAgent library.java.jamm

Review Comment:
   πŸ‘ removed



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on PR #22182:
URL: https://github.com/apache/beam/pull/22182#issuecomment-1191148352

   Thanks so much for your great review @lukecwik , it's been a pleasure :)


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] mosche commented on a diff in pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
mosche commented on code in PR #22182:
URL: https://github.com/apache/beam/pull/22182#discussion_r917907182


##########
sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/schemas/RowBundle.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.beam.sdk.schemas;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toMap;
+import static java.util.stream.IntStream.range;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.sdk.values.RowWithStorage;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.joda.time.DateTime;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+
+/**
+ * Bundle of rows according to the configured {@link Factory} as input for benchmarks.
+ *
+ * <p>The rows are created during {@link #setup()} to exclude initialization costs from the
+ * measurement. To prevent unintended cache hits in {@link RowWithGetters}, a new bundle of rows
+ * must be generated before every invocation.
+ *
+ * <p>Setup per {@link Level#Invocation} has considerable drawbacks. Though, given that processing
+ * bundles of rows (n={@link #bundleSize}) takes well above 1 ms, each individual invocation can be
+ * adequately timestamped without risking generating wrong results.
+ */
+@State(Scope.Benchmark)
+public class RowBundle {
+  @SuppressWarnings("ImmutableEnumChecker") // false positive
+  public enum Action {
+    /**
+     * Write field to object using {@link
+     * GetterBasedSchemaProvider#fromRowFunction(TypeDescriptor)}.
+     *
+     * <p>Use {@link RowWithStorage} to bypass optimizations in RowWithGetters for writes.
+     */
+    WRITE(Factory::createWithStorage),
+
+    /**
+     * Read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_ONCE(Factory::createWithGetter),
+
+    /**
+     * Repeatedly (3x) read field from {@link RowWithGetters} provided by {@link
+     * GetterBasedSchemaProvider#toRowFunction(TypeDescriptor)}.
+     */
+    READ_REPEATED(Factory::createWithGetter);
+
+    final Function<SchemaCoder<?>, Factory<Row>> factoryProvider;
+
+    Action(Function<SchemaCoder<?>, Factory<Row>> factoryProvider) {
+      this.factoryProvider = factoryProvider;
+    }
+  }
+
+  private static final SchemaRegistry REGISTRY = SchemaRegistry.createDefault();
+
+  private final SchemaCoder<?> coder;
+  private final Sink<Row> sink;
+  private Factory<Row> factory;
+
+  protected Row[] rows;
+
+  @Param("100000")
+  int bundleSize;
+
+  @Param({"READ_ONCE", "WRITE"})
+  Action action;
+
+  public RowBundle() {
+    this(null); // unused, just to prevent warnings
+  }

Review Comment:
   This is just to make tooling happy (IDE), @State annotated classes are expected to have a default constructor.



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik commented on PR #22182:
URL: https://github.com/apache/beam/pull/22182#issuecomment-1179260595

   Note that `for` loops are still king over using `IntStream#range` and similar code which might be a trivial fix to get the benchmarks to be dominated by the code under test.


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik merged pull request #22182: JMH module for sdks:java:core with benchmarks for GetterBasedSchemaProvider (resolves #22181)

Posted by GitBox <gi...@apache.org>.
lukecwik merged PR #22182:
URL: https://github.com/apache/beam/pull/22182


-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org