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/05 09:11:06 UTC

[GitHub] [beam] mosche opened a new pull request, #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   Summary of changes:
   - Add tests to verify compatibility with Spark 3.2 and 3.3 (3.1.2 remains the default).
   - Use correct group for Dropwizard `metrics-core`. While package names remained the same, modules have moved to a new group `io.dropwizard.metrics`. This group is also used by Spark and leads to rather indeterministic behavior when loading classes as outdated classes of the old group remain on the classpath.
    -  Recent versions of Dropwizard `MetricRegistry` do not allow registering arbitrary implementations of `Metric`s. Migrate to using `Gauge`.
   - Spark `Sink`s have different constructors between 3.1 and later versions. Using reflection here is fine as there's just one sink per executor.
   - Change encoders/expressions for StructuredStreamingRunner to use reflection where necessary. The overhead of this is rather low as it's just necessary once to build the execution plan.
   - Change StructuredStreamingRunner to allow using a provided / active Spark Sessions for tests. This was critical to fix the broken tests for the metrics integration. On shutdown Spark will stop the metrics system, after that point it's not possible to query any metrics (from the in-memory sink) anymore.
   
   ------------------------
   
   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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?

Review Comment:
   in the default case prefix == "" ?



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkBeamMetricTest.java:
##########
@@ -27,6 +27,7 @@
 
 /** Test SparkBeamMetric. */
 public class SparkBeamMetricTest {

Review Comment:
   done and pushed 👍 



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {
+
+  @Override
+  public final Map<String, Gauge<Double>> getValue() {
+    return getValue("", MetricFilter.ALL);
+  }
+
+  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
+
+  protected Gauge<Double> staticGauge(Number number) {
+    return new StaticGauge(number.doubleValue());
+  }
+
+  private static class StaticGauge implements Gauge<Double> {

Review Comment:
   Intention of that name was a different one here... a Gauge is typical something that pulls a value somewhere, this one is rather static (or better constant) in behavior.  How about one of these: `StaticValue`, `ConstantGauge`, `ConstGauge`, `ConstantValue`, `ConstValue`, `Constant`?



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),
+            Stream.of(name.getNamespace(), name.getName()).map(SparkBeamMetric::cleanPart))
+        .filter(not(Strings::isNullOrEmpty))
+        .collect(Collectors.joining("."));
+  }
 
-    if (step != null) {
-      step = step.replaceAll(ILLEGAL_CHARACTERS, "_");
-      if (step.endsWith("_")) {
-        step = step.substring(0, step.length() - 1);
-      }
-      pieces.add(step);
-    }
+  private static @Nullable String cleanPart(@Nullable String str) {
+    return str != null ? str.replaceAll(ILLEGAL_CHARACTERS, "_") : null;
+  }
 
-    pieces.addAll(
-        ImmutableList.of(name.getNamespace(), name.getName()).stream()
-            .map(str -> str.replaceAll(ILLEGAL_CHARACTERS, "_"))
-            .collect(toList()));
+  private static @Nullable String stripSuffix(@Nullable String str) {
+    return str != null && str.endsWith("_") ? str.substring(0, str.length() - 1) : str;
+  }
 
-    return String.join(".", pieces);
+  private void putFiltered(

Review Comment:
   filters got applied previously, but at the very end after rendering everything. pushing these down just avoids unnecessary work, see https://github.com/apache/beam/pull/22157/files/388512454d4ede037a7b67f2d33d0198a494cff4#diff-91b7484788c580d5c170c9c1add3a7f32fa0a7e2eeb00a6fd4e93db0508bb9c0L119



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java:
##########
@@ -19,256 +19,53 @@
 
 import static org.apache.spark.sql.types.DataTypes.BinaryType;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import org.apache.spark.sql.catalyst.expressions.BoundReference;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.catalyst.expressions.NonSQLExpression;
-import org.apache.spark.sql.catalyst.expressions.UnaryExpression;
-import org.apache.spark.sql.catalyst.expressions.codegen.Block;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext;
-import org.apache.spark.sql.catalyst.expressions.codegen.ExprCode;
+import org.apache.spark.sql.catalyst.expressions.Literal;
 import org.apache.spark.sql.types.DataType;
 import org.apache.spark.sql.types.ObjectType;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import scala.StringContext;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.checkerframework.checker.nullness.qual.NonNull;
 
-/** {@link Encoders} utility class. */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
 public class EncoderHelpers {
+  private static final DataType OBJECT_TYPE = new ObjectType(Object.class);
+
   /**
    * Wrap a Beam coder into a Spark Encoder using Catalyst Expression Encoders (which uses java code
    * generation).
    */
   public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    return EncoderFactory.fromBeamCoder(coder);
+    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
+    // Class T could be private, therefore use OBJECT_TYPE to not risk an IllegalAccessError
+    return EncoderFactory.create(
+        beamSerializer(rootRef(OBJECT_TYPE, true), coder),
+        beamDeserializer(rootCol(BinaryType), coder),

Review Comment:
   :+1: 



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);

Review Comment:
   have you opened the ticket already? If so, please refer it here 



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;

Review Comment:
   yes agree. And moreover I see this continue pattern all over the Beam code base. So it is definitely accepted by the community. I'm not a fan of it, but I will no more mention it



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   R: @aromanenko-dev 
   R: @echauchot 


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);

Review Comment:
   If you prefer using `local[4]` I'm fine changing. That's just a value taken from past experience having worked lots on improving performance of Spark tests. 
   For tests it's generally a misconception that more cores (e.g. all available cores) mean more performance. Tests are typically running in parallel in multiple forks. More cores also means more congestion in that case. Also, using more partitions means more overhead on the processing side (always assuming tests operate on rather tiny datasets). That's why it's important to reduce the number of shuffle partitions for tests before Spark 3.2. (looks like later Spark versions are smarter about this).
   On the other side, using just 1 core / 1 partition can be dangerous as it might obfuscate problems (e.g. invalid assumptions about ordering) in the code. 



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -76,9 +67,10 @@ public void testInBatchMode() throws Exception {
             .apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()))
             .apply(new WordCount.CountWords())
             .apply(MapElements.via(new WordCount.FormatAsTextFn()));
+
     PAssert.that(output).containsInAnyOrder(EXPECTED_COUNTS);
-    pipeline.run();
+    pipeline.run().waitUntilFinish();

Review Comment:
   it's not need, already 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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),
+            Stream.of(name.getNamespace(), name.getName()).map(SparkBeamMetric::cleanPart))
+        .filter(not(Strings::isNullOrEmpty))
+        .collect(Collectors.joining("."));
+  }
 
-    if (step != null) {
-      step = step.replaceAll(ILLEGAL_CHARACTERS, "_");
-      if (step.endsWith("_")) {
-        step = step.substring(0, step.length() - 1);
-      }
-      pieces.add(step);
-    }
+  private static @Nullable String cleanPart(@Nullable String str) {

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] mosche commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;

Review Comment:
   also, that's pretty much personal taste, let's skip such stuff in the future



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java:
##########
@@ -21,14 +21,69 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-public class CodahaleGraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class CodahaleGraphiteSink implements Sink {

Review Comment:
   ok so let's keep this way and avoid breaking change



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/3/build.gradle:
##########
@@ -28,3 +28,37 @@ project.ext {
 
 // Load the main build script which contains all build logic.
 apply from: "$basePath/spark_runner.gradle"
+

Review Comment:
   I like this encapsulation and dynamic testing style



##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);
     return new ExpressionEncoder<>(
         SchemaHelpers.binarySchema(),
         false,
-        JavaConversions.collectionAsScalaIterable(serializers).toSeq(),
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(new GetColumnByOrdinal(0, BinaryType), BinaryType), classTag, coder),
-        classTag);
+        serializers,
+        deserializer,
+        ClassTag$.MODULE$.apply(clazz));
+  }
+
+  static Expression invokeIfNotNull(Class<?> cls, String fun, DataType type, Expression... args) {

Review Comment:
   Missing the javadoc as in the other `EncoderFactory` class.



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);
   }
 
   public SparkSessionRule(String sparkMaster, KV<String, String>... sparkConfig) {
     this(sparkMaster, Arrays.stream(sparkConfig).collect(toMap(KV::getKey, KV::getValue)));
   }
 
+  private static int numDriverCores(String master) {
+    return master.startsWith("local[")
+        ? Integer.parseInt(master.substring("local[".length(), master.length() - 1))

Review Comment:
   duplicated code, please extract in an uility method and deduplicate with `AbstractTranslationContext.java`



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));

Review Comment:
   yes, better to be in sync with what is configured for a prod session



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java:
##########
@@ -35,13 +45,52 @@ public class EncoderHelpersTest {
 
   @ClassRule public static SparkSessionRule sessionRule = new SparkSessionRule();
 
+  private <T> Dataset<T> createDataset(List<T> data, Encoder<T> encoder) {
+    Dataset<T> ds = sessionRule.getSession().createDataset(data, encoder);
+    ds.printSchema();
+    return ds;
+  }
+
   @Test
   public void beamCoderToSparkEncoderTest() {
     List<Integer> data = Arrays.asList(1, 2, 3);
-    Dataset<Integer> dataset =
-        sessionRule
-            .getSession()
-            .createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of()));
+    Dataset<Integer> dataset = createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of()));
     assertEquals(data, dataset.collectAsList());
   }
+
+  @Test
+  public void testBeamEncoderOfPrivateType() {

Review Comment:
   yeah, cf the comment on using OBJECT_TYPE in the prod code.
   
   what was the problem with private types ?



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java:
##########
@@ -35,13 +45,52 @@ public class EncoderHelpersTest {
 
   @ClassRule public static SparkSessionRule sessionRule = new SparkSessionRule();
 
+  private <T> Dataset<T> createDataset(List<T> data, Encoder<T> encoder) {
+    Dataset<T> ds = sessionRule.getSession().createDataset(data, encoder);
+    ds.printSchema();
+    return ds;
+  }
+
   @Test
   public void beamCoderToSparkEncoderTest() {
     List<Integer> data = Arrays.asList(1, 2, 3);
-    Dataset<Integer> dataset =
-        sessionRule
-            .getSession()
-            .createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of()));
+    Dataset<Integer> dataset = createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of()));
     assertEquals(data, dataset.collectAsList());
   }
+
+  @Test
+  public void testBeamEncoderOfPrivateType() {
+    List<PrivateString> data = asList(new PrivateString("1"), new PrivateString("2"));
+    Dataset<PrivateString> dataset = createDataset(data, fromBeamCoder(PrivateString.coder));
+    assertThat(dataset.collect(), equalTo(data.toArray()));
+  }
+
+  private static class PrivateString {
+    private static Coder<PrivateString> coder =

Review Comment:
   final



##########
.test-infra/jenkins/job_PreCommit_Java_Spark3_Versions.groovy:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+import PrecommitJobBuilder
+
+PrecommitJobBuilder builder = new PrecommitJobBuilder(
+    scope: this,
+    nameBase: 'Java_Spark3_Versions',
+    gradleTask: ':runners:spark:3:sparkVersionsTest',
+    gradleSwitches: [
+      '-PdisableSpotlessCheck=true'

Review Comment:
   we should not disable spotless check



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,33 +17,48 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import java.lang.reflect.Constructor;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag;
-import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
+  // default constructor to reflectively create static invoke expressions
+  private static final Constructor<StaticInvoke> STATIC_INVOKE_CONSTRUCTOR =
+      (Constructor<StaticInvoke>) StaticInvoke.class.getConstructors()[0];
+
+  static <T> ExpressionEncoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    return new ExpressionEncoder<>(serializer, deserializer, ClassTag.apply(clazz));
+  }
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    Expression serializer =
-        new EncoderHelpers.EncodeUsingBeamCoder<>(
-            new BoundReference(0, new ObjectType(clazz), true), coder);
-    Expression deserializer =
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(
-                new GetColumnByOrdinal(0, BinaryType), BinaryType, scala.Option.<String>empty()),
-            classTag,
-            coder);
-    return new ExpressionEncoder<>(serializer, deserializer, classTag);
+  /**
+   * Invoke method {@code fun} on Class {@code cls}, immediately propagating {@code null} if any
+   * input arg is {@code null}.
+   *
+   * <p>To address breaking interfaces between various version of Spark 3 these are created

Review Comment:
   you could have also passed the spark version in (from the spark session) and call the correct constructor depending on the version. That way you would have spared using the reflexion api and also guessing the version based on the number of parameters of the constructor



##########
runners/spark/3/build.gradle:
##########
@@ -28,3 +28,37 @@ project.ext {
 
 // Load the main build script which contains all build logic.
 apply from: "$basePath/spark_runner.gradle"
+
+
+def sparkVersions = [
+    "330": "3.3.0",
+    "321": "3.2.1"
+]
+
+sparkVersions.each { kv ->

Review Comment:
   ok, you test on all supported versions, but what version is the resulting spark-3-runner artifact built against ?



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);

Review Comment:
   default to 4 is the value people are used to. Why change or why not let it detect the number of CPU cores of the system ?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java:
##########
@@ -146,10 +146,12 @@ public SparkStructuredStreamingPipelineResult run(final Pipeline pipeline) {
             });
     executorService.shutdown();
 
-    // TODO: Streaming.
+    Runnable onTerminalState =
+        options.getUseActiveSparkSession()
+            ? () -> {}
+            : () -> translationContext.getSparkSession().stop();

Review Comment:
   Compared to what old SparkStructuredStreamingPipelineResult#stop(() did, this callback misses to set the pipeline state: `if the pipeline was running, then set its state to State.STOPPED`



##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);

Review Comment:
   I'm sorry but I find this line unreadable (mainly because of scala integration in java). This is only to add an element at the beginning of an empty list. I would prefer `JavaConversions.collectionAsScalaIterable()` and make the collection in java. Remember that there is no scala in Beam so we should reduce scala to the strict minimum.



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);
   }
 
   public SparkSessionRule(String sparkMaster, KV<String, String>... sparkConfig) {
     this(sparkMaster, Arrays.stream(sparkConfig).collect(toMap(KV::getKey, KV::getValue)));
   }
 
+  private static int numDriverCores(String master) {

Review Comment:
   please rename into `extractNumWorkers`.



##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);
     return new ExpressionEncoder<>(
         SchemaHelpers.binarySchema(),
         false,
-        JavaConversions.collectionAsScalaIterable(serializers).toSeq(),
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(new GetColumnByOrdinal(0, BinaryType), BinaryType), classTag, coder),
-        classTag);
+        serializers,
+        deserializer,
+        ClassTag$.MODULE$.apply(clazz));
+  }
+
+  static Expression invokeIfNotNull(Class<?> cls, String fun, DataType type, Expression... args) {
+    return new StaticInvoke(cls, type, fun, seqOf(args), true, true);
+  }
+
+  private static Seq<Expression> seqOf(Expression... args) {

Review Comment:
   inline like in the other `EncoderFactory` class



##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);
     return new ExpressionEncoder<>(
         SchemaHelpers.binarySchema(),
         false,
-        JavaConversions.collectionAsScalaIterable(serializers).toSeq(),
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(new GetColumnByOrdinal(0, BinaryType), BinaryType), classTag, coder),
-        classTag);
+        serializers,
+        deserializer,
+        ClassTag$.MODULE$.apply(clazz));

Review Comment:
   Can't you just use `ClassTag.apply(clazz)` as in the other EncoderFactory class ?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java:
##########
@@ -19,256 +19,53 @@
 
 import static org.apache.spark.sql.types.DataTypes.BinaryType;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import org.apache.spark.sql.catalyst.expressions.BoundReference;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.catalyst.expressions.NonSQLExpression;
-import org.apache.spark.sql.catalyst.expressions.UnaryExpression;
-import org.apache.spark.sql.catalyst.expressions.codegen.Block;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext;
-import org.apache.spark.sql.catalyst.expressions.codegen.ExprCode;
+import org.apache.spark.sql.catalyst.expressions.Literal;
 import org.apache.spark.sql.types.DataType;
 import org.apache.spark.sql.types.ObjectType;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import scala.StringContext;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.checkerframework.checker.nullness.qual.NonNull;
 
-/** {@link Encoders} utility class. */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
 public class EncoderHelpers {
+  private static final DataType OBJECT_TYPE = new ObjectType(Object.class);
+
   /**
    * Wrap a Beam coder into a Spark Encoder using Catalyst Expression Encoders (which uses java code
    * generation).
    */
   public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    return EncoderFactory.fromBeamCoder(coder);
+    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
+    // Class T could be private, therefore use OBJECT_TYPE to not risk an IllegalAccessError
+    return EncoderFactory.create(
+        beamSerializer(rootRef(OBJECT_TYPE, true), coder),
+        beamDeserializer(rootCol(BinaryType), coder),

Review Comment:
   Thanks for that ! It is way more maintainable than relying on code generation ! What I wonder is why spark bundled Encoders like Kryo rely on code generation. Have you seen a perf drop of not using code generation in Beam ExpressionEncoders ?



##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,33 +17,48 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import java.lang.reflect.Constructor;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag;
-import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
+  // default constructor to reflectively create static invoke expressions
+  private static final Constructor<StaticInvoke> STATIC_INVOKE_CONSTRUCTOR =
+      (Constructor<StaticInvoke>) StaticInvoke.class.getConstructors()[0];
+
+  static <T> ExpressionEncoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    return new ExpressionEncoder<>(serializer, deserializer, ClassTag.apply(clazz));

Review Comment:
   it is more logical API than what was in v2 where there was a list of serializers but a single deserializer



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/sink/CsvSink.java:
##########
@@ -18,22 +18,64 @@
 package org.apache.beam.runners.spark.metrics.sink;
 
 import com.codahale.metrics.MetricRegistry;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.lang.reflect.Constructor;
 import java.util.Properties;
 import org.apache.beam.runners.spark.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.metrics.WithMetricsSupport;
 import org.apache.spark.metrics.sink.Sink;
 
 /**
- * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to a CSV file.
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.metrics.sink.CsvSink"
+ * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
+ * "spark.metrics.conf.*.sink.csv.period"=10
+ * "spark.metrics.conf.*.sink.csv.unit"=seconds
+ * }</pre>
  */
-// Intentionally overriding parent name because inheritors should replace the parent.
-@SuppressFBWarnings("NM_SAME_SIMPLE_NAME_AS_SUPERCLASS")
-public class CsvSink extends org.apache.spark.metrics.sink.CsvSink {
+public class CsvSink implements Sink {
+
+  // Initialized reflectively as done by Spark's MetricsSystem
+  private final org.apache.spark.metrics.sink.CsvSink delegate;
+
+  /** Constructor for Spark 3.1.x. */
   public CsvSink(
       final Properties properties,
       final MetricRegistry metricRegistry,
       final org.apache.spark.SecurityManager securityMgr) {
-    super(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+  }
+
+  /** Constructor for Spark 3.2.x and later. */
+  public CsvSink(final Properties properties, final MetricRegistry metricRegistry) {
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry));
+  }
+
+  @Override
+  public void start() {
+    delegate.start();
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+
+  @Override
+  public void report() {
+    delegate.report();
+  }
+
+  private static org.apache.spark.metrics.sink.CsvSink newDelegate(Object... params) {
+    try {
+      Constructor<?> constructor = org.apache.spark.metrics.sink.CsvSink.class.getConstructors()[0];

Review Comment:
   I'm not a big fan of using the reflexion API. I know why you do it, it is to avoid having 2 maven modules for spark 3.1 and spark 3.2+. So you don't know which is your runtime spark version so you need to use the reflexion api. But I would prefer that you use an explicit call to the correct constructor like below rather than using varargs and constructor[0] which could change in the future without compilation error.
   ```
    delegate = org.apache.spark.metrics.sink.CsvSink.class.getConstructor(Properties.class,
         MetricRegistry.class, org.apache.spark.SecurityManager.class).newInstance(properties, metricRegistry, securityMgr);
   ```



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleCsvSink.java:
##########
@@ -18,19 +18,64 @@
 package org.apache.beam.runners.spark.structuredstreaming.metrics.sink;
 
 import com.codahale.metrics.MetricRegistry;
+import java.lang.reflect.Constructor;
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
 import org.apache.spark.metrics.sink.Sink;
 
 /**
- * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to a CSV file.
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleCsvSink"
+ * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
+ * "spark.metrics.conf.*.sink.csv.period"=10
+ * "spark.metrics.conf.*.sink.csv.unit"=seconds
+ * }</pre>
  */
-public class CodahaleCsvSink extends org.apache.spark.metrics.sink.CsvSink {
+public class CodahaleCsvSink implements Sink {
+
+  // Initialized reflectively as done by Spark's MetricsSystem
+  private final org.apache.spark.metrics.sink.CsvSink delegate;
+
+  /** Constructor for Spark 3.1.x. */
   public CodahaleCsvSink(
       final Properties properties,
       final MetricRegistry metricRegistry,
       final org.apache.spark.SecurityManager securityMgr) {
-    super(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+  }
+
+  /** Constructor for Spark 3.2.x and later. */
+  public CodahaleCsvSink(final Properties properties, final MetricRegistry metricRegistry) {
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry));
+  }
+
+  @Override
+  public void start() {
+    delegate.start();
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+
+  @Override
+  public void report() {
+    delegate.report();
+  }
+
+  private static org.apache.spark.metrics.sink.CsvSink newDelegate(Object... params) {
+    try {
+      Constructor<?> constructor = org.apache.spark.metrics.sink.CsvSink.class.getConstructors()[0];

Review Comment:
   ditto



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/sink/GraphiteSink.java:
##########
@@ -18,20 +18,68 @@
 package org.apache.beam.runners.spark.metrics.sink;
 
 import com.codahale.metrics.MetricRegistry;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.lang.reflect.Constructor;
 import java.util.Properties;
 import org.apache.beam.runners.spark.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.metrics.WithMetricsSupport;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-// Intentionally overriding parent name because inheritors should replace the parent.
-@SuppressFBWarnings("NM_SAME_SIMPLE_NAME_AS_SUPERCLASS")
-public class GraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.metrics.sink.GraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class GraphiteSink implements Sink {
+
+  // Initialized reflectively as done by Spark's MetricsSystem
+  private final org.apache.spark.metrics.sink.GraphiteSink delegate;
+
+  /** Constructor for Spark 3.1.x. */
   public GraphiteSink(
       final Properties properties,
       final MetricRegistry metricRegistry,
       final org.apache.spark.SecurityManager securityMgr) {
-    super(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+  }
+
+  /** Constructor for Spark 3.2.x and later. */
+  public GraphiteSink(final Properties properties, final MetricRegistry metricRegistry) {
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry));
+  }
+
+  @Override
+  public void start() {
+    delegate.start();
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+
+  @Override
+  public void report() {
+    delegate.report();
+  }
+
+  private static org.apache.spark.metrics.sink.GraphiteSink newDelegate(Object... params) {
+    try {
+      Constructor<?> constructor =
+          org.apache.spark.metrics.sink.GraphiteSink.class.getConstructors()[0];

Review Comment:
   same comment as for CsvSink



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java:
##########
@@ -18,17 +18,68 @@
 package org.apache.beam.runners.spark.structuredstreaming.metrics.sink;
 
 import com.codahale.metrics.MetricRegistry;
+import java.lang.reflect.Constructor;
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-public class CodahaleGraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class CodahaleGraphiteSink implements Sink {
+
+  // Initialized reflectively as done by Spark's MetricsSystem
+  private final org.apache.spark.metrics.sink.GraphiteSink delegate;
+
+  /** Constructor for Spark 3.1.x. */
   public CodahaleGraphiteSink(
       final Properties properties,
       final MetricRegistry metricRegistry,
       final org.apache.spark.SecurityManager securityMgr) {
-    super(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+  }
+
+  /** Constructor for Spark 3.2.x and later. */
+  public CodahaleGraphiteSink(final Properties properties, final MetricRegistry metricRegistry) {
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry));
+  }
+
+  @Override
+  public void start() {
+    delegate.start();
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+
+  @Override
+  public void report() {
+    delegate.report();
+  }
+
+  private static org.apache.spark.metrics.sink.GraphiteSink newDelegate(Object... params) {
+    try {
+      Constructor<?> constructor =
+          org.apache.spark.metrics.sink.GraphiteSink.class.getConstructors()[0];

Review Comment:
   ditto



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/sink/CsvSink.java:
##########
@@ -18,22 +18,64 @@
 package org.apache.beam.runners.spark.metrics.sink;
 
 import com.codahale.metrics.MetricRegistry;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.lang.reflect.Constructor;
 import java.util.Properties;
 import org.apache.beam.runners.spark.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.metrics.WithMetricsSupport;
 import org.apache.spark.metrics.sink.Sink;
 
 /**
- * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to a CSV file.
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.metrics.sink.CsvSink"
+ * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
+ * "spark.metrics.conf.*.sink.csv.period"=10
+ * "spark.metrics.conf.*.sink.csv.unit"=seconds
+ * }</pre>
  */
-// Intentionally overriding parent name because inheritors should replace the parent.
-@SuppressFBWarnings("NM_SAME_SIMPLE_NAME_AS_SUPERCLASS")
-public class CsvSink extends org.apache.spark.metrics.sink.CsvSink {
+public class CsvSink implements Sink {
+
+  // Initialized reflectively as done by Spark's MetricsSystem
+  private final org.apache.spark.metrics.sink.CsvSink delegate;
+
+  /** Constructor for Spark 3.1.x. */
   public CsvSink(
       final Properties properties,
       final MetricRegistry metricRegistry,
       final org.apache.spark.SecurityManager securityMgr) {
-    super(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+  }
+
+  /** Constructor for Spark 3.2.x and later. */
+  public CsvSink(final Properties properties, final MetricRegistry metricRegistry) {
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry));
+  }
+
+  @Override
+  public void start() {
+    delegate.start();
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+
+  @Override
+  public void report() {
+    delegate.report();
+  }
+
+  private static org.apache.spark.metrics.sink.CsvSink newDelegate(Object... params) {
+    try {
+      Constructor<?> constructor = org.apache.spark.metrics.sink.CsvSink.class.getConstructors()[0];

Review Comment:
   :+1: 



-- 
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 merged pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


-- 
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] echauchot commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   reviewing ...


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),

Review Comment:
   but what's the value of keeping the step variable?



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);

Review Comment:
   was just mentioning this as it is the custom for beam users. But I'm not sure it will entails any trouble for them. And I agree with the arguments you gave I'd add that local[1] could also obfuscate serialization issues. Fair enough for local[2]



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);
   }
 
   public SparkSessionRule(String sparkMaster, KV<String, String>... sparkConfig) {
     this(sparkMaster, Arrays.stream(sparkConfig).collect(toMap(KV::getKey, KV::getValue)));
   }
 
+  private static int numDriverCores(String master) {

Review Comment:
   +1 on the factory, I just looked at it



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);

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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)

Review Comment:
   no the RunWith annotation I think



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleCsvSink.java:
##########
@@ -21,16 +21,66 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
 /**
- * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to a CSV file.
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleCsvSink"
+ * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
+ * "spark.metrics.conf.*.sink.csv.period"=10
+ * "spark.metrics.conf.*.sink.csv.unit"=seconds
+ * }</pre>
  */
-public class CodahaleCsvSink extends org.apache.spark.metrics.sink.CsvSink {
+public class CodahaleCsvSink implements Sink {

Review Comment:
   I don't wanna introduce any breaking changes in this PR



-- 
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] echauchot commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   Run Spark StructuredStreaming ValidatesRunner


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {
+
+  @Override
+  public final Map<String, Gauge<Double>> getValue() {
+    return getValue("", MetricFilter.ALL);
+  }
+
+  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
+
+  protected Gauge<Double> staticGauge(Number number) {
+    return new StaticGauge(number.doubleValue());
+  }
+
+  private static class StaticGauge implements Gauge<Double> {
+    double value;

Review Comment:
   done



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {
+
+  @Override
+  public final Map<String, Gauge<Double>> getValue() {
+    return getValue("", MetricFilter.ALL);
+  }
+
+  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
+
+  protected Gauge<Double> staticGauge(Number number) {
+    return new StaticGauge(number.doubleValue());
+  }
+
+  private static class StaticGauge implements Gauge<Double> {
+    double value;

Review Comment:
   it is indeed even clearer with the final that we deal with a constant gauge



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),
+            Stream.of(name.getNamespace(), name.getName()).map(SparkBeamMetric::cleanPart))
+        .filter(not(Strings::isNullOrEmpty))
+        .collect(Collectors.joining("."));
+  }
 
-    if (step != null) {
-      step = step.replaceAll(ILLEGAL_CHARACTERS, "_");
-      if (step.endsWith("_")) {
-        step = step.substring(0, step.length() - 1);
-      }
-      pieces.add(step);
-    }
+  private static @Nullable String cleanPart(@Nullable String str) {
+    return str != null ? str.replaceAll(ILLEGAL_CHARACTERS, "_") : null;
+  }
 
-    pieces.addAll(
-        ImmutableList.of(name.getNamespace(), name.getName()).stream()
-            .map(str -> str.replaceAll(ILLEGAL_CHARACTERS, "_"))
-            .collect(toList()));
+  private static @Nullable String stripSuffix(@Nullable String str) {
+    return str != null && str.endsWith("_") ? str.substring(0, str.length() - 1) : str;
+  }
 
-    return String.join(".", pieces);
+  private void putFiltered(

Review Comment:
   yes, Once again I saw the `WithMetricsSupport` class after this 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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),

Review Comment:
   so that code readers could match this code with the functional **step** mentioned in Beam documentation.



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java:
##########
@@ -19,256 +19,53 @@
 
 import static org.apache.spark.sql.types.DataTypes.BinaryType;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import org.apache.spark.sql.catalyst.expressions.BoundReference;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.catalyst.expressions.NonSQLExpression;
-import org.apache.spark.sql.catalyst.expressions.UnaryExpression;
-import org.apache.spark.sql.catalyst.expressions.codegen.Block;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext;
-import org.apache.spark.sql.catalyst.expressions.codegen.ExprCode;
+import org.apache.spark.sql.catalyst.expressions.Literal;
 import org.apache.spark.sql.types.DataType;
 import org.apache.spark.sql.types.ObjectType;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import scala.StringContext;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.checkerframework.checker.nullness.qual.NonNull;
 
-/** {@link Encoders} utility class. */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
 public class EncoderHelpers {
+  private static final DataType OBJECT_TYPE = new ObjectType(Object.class);
+
   /**
    * Wrap a Beam coder into a Spark Encoder using Catalyst Expression Encoders (which uses java code
    * generation).
    */
   public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    return EncoderFactory.fromBeamCoder(coder);
+    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
+    // Class T could be private, therefore use OBJECT_TYPE to not risk an IllegalAccessError
+    return EncoderFactory.create(
+        beamSerializer(rootRef(OBJECT_TYPE, true), coder),
+        beamDeserializer(rootCol(BinaryType), coder),

Review Comment:
   and no perf drop noticed to use the higher level API ?



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {
+
+  @Override
+  public final Map<String, Gauge<Double>> getValue() {
+    return getValue("", MetricFilter.ALL);
+  }
+
+  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
+
+  protected Gauge<Double> staticGauge(Number number) {
+    return new StaticGauge(number.doubleValue());
+  }
+
+  private static class StaticGauge implements Gauge<Double> {
+    double value;

Review Comment:
   yeah, missed that one



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java:
##########
@@ -21,14 +21,69 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-public class CodahaleGraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class CodahaleGraphiteSink implements Sink {

Review Comment:
   need to think. It is not very user facing except for configuration like "my sink = my class"
   



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?

Review Comment:
   it's the same as before to avoid any breaking change ...



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   Thanks a lot for the first round of feedback @echauchot, I've pushed related changes.


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);

Review Comment:
   I agree it's ugly and painful to interface with Scala from Java. BUT that's also the nature of dealing with Spark at a lower level beyond what's offered on the Java API to create an efficient runner :(
   What I'd suggest is to create a single static utility `ScalaInterop(erability)` to deal with these. But I'd rather not do it in this PR as I'm running into more and more conflicts with the pending PR for the structured streaming runner.
    



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }

Review Comment:
   very controversial topic, but i agree with you... :)



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;

Review Comment:
   yes agree. And moreover I see this continue pattern all over the Beam code base. So it is definitely accepted by the community. I'm not a fan of it, it is ok then.



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();

Review Comment:
   yes, I reviewed the `WithMetricsSupport` after I wrote this comment and I forgot to update it.



-- 
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] echauchot commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   > > as a general advice, it is easier for review process if you do tiny commits that address one comment (we will squash anyway) that way I can track the addressed comments and resolve the conversations myself
   > 
   > fair comment @echauchot, but I guess we have to meet somewhere in the middle ... that would make addressing comments more than a hassle
   
   yes somewere in the middle looks good to me. What I advice is what I do when I'm being reviewed but it is true that sometimes I commit more than one change and it requires that I split the commit afterwards. Painful ...


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?

Review Comment:
   yes and no, usage is always as follows (`prefix = entry.getKey()`)
   ```
   ((BeamMetricSet) gauge).getValue(entry.getKey(), filter)
   ```



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java:
##########
@@ -21,14 +21,69 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-public class CodahaleGraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class CodahaleGraphiteSink implements Sink {

Review Comment:
   well, still a breaking change (probably even worse as this will cause it to fail silently at runtime)



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
.test-infra/jenkins/job_PreCommit_Java_Spark3_Versions.groovy:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+import PrecommitJobBuilder
+
+PrecommitJobBuilder builder = new PrecommitJobBuilder(
+    scope: this,
+    nameBase: 'Java_Spark3_Versions',
+    gradleTask: ':runners:spark:3:sparkVersionsTest',
+    gradleSwitches: [
+      '-PdisableSpotlessCheck=true'

Review Comment:
   This is the additional tests only, spotless is already checked as part of the "normal" test run. No point in running that again...Btw, that's the same as the hadoop version tests



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);
   }
 
   public SparkSessionRule(String sparkMaster, KV<String, String>... sparkConfig) {
     this(sparkMaster, Arrays.stream(sparkConfig).collect(toMap(KV::getKey, KV::getValue)));
   }
 
+  private static int numDriverCores(String master) {

Review Comment:
   This has become obsolete with above comment, introduced a `SparkSessionFactory`. Btw, that's the Spark terminology... see `SparkContext.numDriverCores(String master, ...)`



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);
     return new ExpressionEncoder<>(
         SchemaHelpers.binarySchema(),
         false,
-        JavaConversions.collectionAsScalaIterable(serializers).toSeq(),
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(new GetColumnByOrdinal(0, BinaryType), BinaryType), classTag, coder),
-        classTag);
+        serializers,
+        deserializer,
+        ClassTag$.MODULE$.apply(clazz));

Review Comment:
   Unfortunately not, that's a breaking change in the Scala version



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/3/build.gradle:
##########
@@ -28,3 +28,37 @@ project.ext {
 
 // Load the main build script which contains all build logic.
 apply from: "$basePath/spark_runner.gradle"
+
+
+def sparkVersions = [
+    "330": "3.3.0",
+    "321": "3.2.1"
+]
+
+sparkVersions.each { kv ->

Review Comment:
   exactly... that's pretty much how things look like in real life. you run a job build with one version of beam on a spark cluster, but the spark versions the runner was compiled against and the cluster don't necessarily match



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),

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] mosche commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java:
##########
@@ -19,256 +19,53 @@
 
 import static org.apache.spark.sql.types.DataTypes.BinaryType;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import org.apache.spark.sql.catalyst.expressions.BoundReference;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.catalyst.expressions.NonSQLExpression;
-import org.apache.spark.sql.catalyst.expressions.UnaryExpression;
-import org.apache.spark.sql.catalyst.expressions.codegen.Block;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext;
-import org.apache.spark.sql.catalyst.expressions.codegen.ExprCode;
+import org.apache.spark.sql.catalyst.expressions.Literal;
 import org.apache.spark.sql.types.DataType;
 import org.apache.spark.sql.types.ObjectType;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import scala.StringContext;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.checkerframework.checker.nullness.qual.NonNull;
 
-/** {@link Encoders} utility class. */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
 public class EncoderHelpers {
+  private static final DataType OBJECT_TYPE = new ObjectType(Object.class);
+
   /**
    * Wrap a Beam coder into a Spark Encoder using Catalyst Expression Encoders (which uses java code
    * generation).
    */
   public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    return EncoderFactory.fromBeamCoder(coder);
+    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
+    // Class T could be private, therefore use OBJECT_TYPE to not risk an IllegalAccessError
+    return EncoderFactory.create(
+        beamSerializer(rootRef(OBJECT_TYPE, true), coder),
+        beamDeserializer(rootCol(BinaryType), coder),

Review Comment:
   This is just using more higher level expressions to make the code more maintainable. Spark catalyst expressions rely on code generation underneath, no way to avoid that ... :/



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   Hmm, looks like license headers for shared sources aren't added automatically ... 


-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }

Review Comment:
   remove final in the parameters as in the other aggregatormetric class.
   I agreed with this suppression: final parameters are kind of a convention in java. Otherwise we would have final parameters in all methods.



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {

Review Comment:
   can be package local



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),

Review Comment:
   keep the step variable because it has a meaning for beam (step in the pipeline, ie transform) cf [metrics pres](https://www.slideshare.net/slideshow/embed_code/key/kKJRzR8HxkxLsR). cf s17 and s26



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),
+            Stream.of(name.getNamespace(), name.getName()).map(SparkBeamMetric::cleanPart))
+        .filter(not(Strings::isNullOrEmpty))
+        .collect(Collectors.joining("."));

Review Comment:
   clearer than previous code, thanks !



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java:
##########
@@ -39,26 +42,24 @@ public InMemoryMetrics(
     internalMetricRegistry = metricRegistry;
   }
 
-  @SuppressWarnings("TypeParameterUnusedInFormals")
-  public static <T> T valueOf(final String name) {
-    final T retVal;
+  // Constructor for Spark >= 3.2
+  @SuppressWarnings("UnusedParameters")
+  public InMemoryMetrics(final Properties properties, final MetricRegistry metricRegistry) {
+    extendedMetricsRegistry = WithMetricsSupport.forRegistry(metricRegistry);
+    internalMetricRegistry = metricRegistry;
+  }
 
+  @SuppressWarnings({"TypeParameterUnusedInFormals", "rawtypes"})
+  public static <T> T valueOf(final String name) {
     // this might fail in case we have multiple aggregators with the same suffix after
     // the last dot, but it should be good enough for tests.
-    if (extendedMetricsRegistry != null
-        && extendedMetricsRegistry.getGauges().keySet().stream()
-            .anyMatch(Predicates.containsPattern(name + "$")::apply)) {
-      String key =
-          extendedMetricsRegistry.getGauges().keySet().stream()
-              .filter(Predicates.containsPattern(name + "$")::apply)
-              .findFirst()
-              .get();
-      retVal = (T) extendedMetricsRegistry.getGauges().get(key).getValue();
+    if (extendedMetricsRegistry != null) {
+      Collection<Gauge> matches =
+          extendedMetricsRegistry.getGauges((n, m) -> n.endsWith(name)).values();

Review Comment:
   simpler code thanks



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -76,9 +67,10 @@ public void testInBatchMode() throws Exception {
             .apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()))
             .apply(new WordCount.CountWords())
             .apply(MapElements.via(new WordCount.FormatAsTextFn()));
+
     PAssert.that(output).containsInAnyOrder(EXPECTED_COUNTS);
-    pipeline.run();
+    pipeline.run().waitUntilFinish();

Review Comment:
   IIRC in test mode the pipeline default used to be waitUntilFish so that we have a blocking call to assert afterwards. Is it not still the case ?



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkBeamMetricTest.java:
##########
@@ -27,6 +27,7 @@
 
 /** Test SparkBeamMetric. */
 public class SparkBeamMetricTest {

Review Comment:
   why is there no SparkBeamMetricTest for the SS runner ? Ah I get it it is the BeamMetricTest  class. Can you rename BeamMetricTest to SparkBeamMetricTest as the class it tests is SparkBeamMetric ?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.

Review Comment:
   ```suggestion
    * Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
    * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
    * Metrics}. So this class is a Beam metrics registry implemented as a Dropwizard {@link Gauge} for use by spark engine.
   ```



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();

Review Comment:
   So now, all the beam metrics are stored in dropwizard `Gauge<Double>` ?  You're sure that all the types of Beam metrics `Counter, Gauge, Histogram and Distribution` are compatible with DropWizard `Gauge<Double>`. I guess as Distribution content is extracted into 5 entries as before it works. We will see in the tests anyway.
   
   But still no support for histograms (though it was not the aim of the PR to add this feature)



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?

Review Comment:
   breaking change ? Users will have their spark metrics prefixed now or is this abstracted by new spark engine ?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {
+
+  @Override
+  public final Map<String, Gauge<Double>> getValue() {
+    return getValue("", MetricFilter.ALL);
+  }
+
+  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
+
+  protected Gauge<Double> staticGauge(Number number) {
+    return new StaticGauge(number.doubleValue());
+  }
+
+  private static class StaticGauge implements Gauge<Double> {

Review Comment:
   we always make nested class static when possible to avoid serialization issues. So this name is not very relevant consider renaming to `DoubleGauge`



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.structuredstreaming.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {

Review Comment:
   can be package local



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),
+            Stream.of(name.getNamespace(), name.getName()).map(SparkBeamMetric::cleanPart))
+        .filter(not(Strings::isNullOrEmpty))
+        .collect(Collectors.joining("."));
+  }
 
-    if (step != null) {
-      step = step.replaceAll(ILLEGAL_CHARACTERS, "_");
-      if (step.endsWith("_")) {
-        step = step.substring(0, step.length() - 1);
-      }
-      pieces.add(step);
-    }
+  private static @Nullable String cleanPart(@Nullable String str) {

Review Comment:
   nit picking: I would rename to `normalizeString` or something similar



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;

Review Comment:
   I'm not a big fan of this coding style. I would prefer 
   ```
   if (rawValue  != null) {
   try {
   ...
   }
   }
   ```



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java:
##########
@@ -21,14 +21,69 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-public class CodahaleGraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class CodahaleGraphiteSink implements Sink {

Review Comment:
   Why not call it GraphiteSink like in the non-structuredstreaming runner ?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),
+            Stream.of(name.getNamespace(), name.getName()).map(SparkBeamMetric::cleanPart))
+        .filter(not(Strings::isNullOrEmpty))
+        .collect(Collectors.joining("."));
+  }
 
-    if (step != null) {
-      step = step.replaceAll(ILLEGAL_CHARACTERS, "_");
-      if (step.endsWith("_")) {
-        step = step.substring(0, step.length() - 1);
-      }
-      pieces.add(step);
-    }
+  private static @Nullable String cleanPart(@Nullable String str) {
+    return str != null ? str.replaceAll(ILLEGAL_CHARACTERS, "_") : null;
+  }
 
-    pieces.addAll(
-        ImmutableList.of(name.getNamespace(), name.getName()).stream()
-            .map(str -> str.replaceAll(ILLEGAL_CHARACTERS, "_"))
-            .collect(toList()));
+  private static @Nullable String stripSuffix(@Nullable String str) {
+    return str != null && str.endsWith("_") ? str.substring(0, str.length() - 1) : str;
+  }
 
-    return String.join(".", pieces);
+  private void putFiltered(

Review Comment:
   previously the metrics were not filtered, how is it working with the dropWizard filters ?



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;
+      }
+      try {
+        Gauge<Double> gauge = staticGauge(rawValue);
+        if (filter.matches(name, gauge)) {
+          metrics.put(name, gauge);
+        }
+      } catch (NumberFormatException e) {
+        LOG.warn(
+            "Metric `{}` of type {} can't be reported, conversion to double failed.",
+            name,
+            rawValue.getClass().getSimpleName(),
+            e);
+      }
+    }
+    return metrics;
+  }
+
+  // Metric type is assumed to be compatible with Double

Review Comment:
   let me get this clear: with new DropWizard version you can no more create a custom metrics registry so you use a DropWizard Gauge to make a registry (`Gauge<Map<String, Gauge<Double>>>`).  Every type of Beam metrics can be stored in DropWizard `Gauge<Double>`. 
   
   See also similar comment in the other subclass of  BeamMetricSet



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/WithMetricsSupport.java:
##########
@@ -88,95 +71,21 @@ public SortedMap<String, Counter> getCounters(final MetricFilter filter) {
 
   @Override
   public SortedMap<String, Gauge> getGauges(final MetricFilter filter) {
-    return new ImmutableSortedMap.Builder<String, Gauge>(
-            Ordering.from(String.CASE_INSENSITIVE_ORDER))
-        .putAll(internalMetricRegistry.getGauges(filter))
-        .putAll(extractGauges(internalMetricRegistry, filter))
-        .build();
-  }
-
-  private Map<String, Gauge> extractGauges(
-      final MetricRegistry metricRegistry, final MetricFilter filter) {
-    Map<String, Gauge> gauges = new HashMap<>();
-
-    // find the AggregatorMetric metrics from within all currently registered metrics
-    final Optional<Map<String, Gauge>> aggregatorMetrics =
-        FluentIterable.from(metricRegistry.getMetrics().entrySet())
-            .firstMatch(isAggregatorMetric())
-            .transform(aggregatorMetricToGauges());
-
-    // find the SparkBeamMetric metrics from within all currently registered metrics
-    final Optional<Map<String, Gauge>> beamMetrics =
-        FluentIterable.from(metricRegistry.getMetrics().entrySet())
-            .firstMatch(isSparkBeamMetric())
-            .transform(beamMetricToGauges());
-
-    if (aggregatorMetrics.isPresent()) {
-      gauges.putAll(Maps.filterEntries(aggregatorMetrics.get(), matches(filter)));
-    }
-
-    if (beamMetrics.isPresent()) {
-      gauges.putAll(Maps.filterEntries(beamMetrics.get(), matches(filter)));
-    }
-
-    return gauges;
-  }
-
-  private Function<Map.Entry<String, Metric>, Map<String, Gauge>> aggregatorMetricToGauges() {
-    return entry -> {
-      final NamedAggregators agg = ((AggregatorMetric) entry.getValue()).getNamedAggregators();
-      final String parentName = entry.getKey();
-      final Map<String, Gauge> gaugeMap = Maps.transformEntries(agg.renderAll(), toGauge());
-      final Map<String, Gauge> fullNameGaugeMap = Maps.newLinkedHashMap();
-      for (Map.Entry<String, Gauge> gaugeEntry : gaugeMap.entrySet()) {
-        fullNameGaugeMap.put(parentName + "." + gaugeEntry.getKey(), gaugeEntry.getValue());
+    ImmutableSortedMap.Builder<String, Gauge> builder =
+        new ImmutableSortedMap.Builder<>(Ordering.from(String.CASE_INSENSITIVE_ORDER));
+
+    Map<String, Gauge> gauges =
+        internalMetricRegistry.getGauges(
+            (n, m) -> filter.matches(n, m) || m instanceof BeamMetricSet);
+
+    for (Map.Entry<String, Gauge> entry : gauges.entrySet()) {
+      Gauge gauge = entry.getValue();
+      if (gauge instanceof BeamMetricSet) {
+        builder.putAll(((BeamMetricSet) gauge).getValue(entry.getKey(), filter));
+      } else {
+        builder.put(entry.getKey(), gauge);

Review Comment:
   Way more maintainable code, thanks ! 
   If the tests cover enough and pass, we are all good



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)
 public class SparkMetricsSinkTest {
+
+  @ClassRule
+  public static SparkSessionRule sessionRule =
+      new SparkSessionRule(
+          KV.of("spark.metrics.conf.*.sink.memory.class", InMemoryMetrics.class.getName()));

Review Comment:
   Better indeed to configure the context here rather than having the strange conf file with a single entry as before. Thanks



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleCsvSink.java:
##########
@@ -21,16 +21,66 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
 /**
- * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to a CSV file.
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleCsvSink"
+ * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
+ * "spark.metrics.conf.*.sink.csv.period"=10
+ * "spark.metrics.conf.*.sink.csv.unit"=seconds
+ * }</pre>
  */
-public class CodahaleCsvSink extends org.apache.spark.metrics.sink.CsvSink {
+public class CodahaleCsvSink implements Sink {

Review Comment:
   Why not call it CsvSink like in the non-structuredstreaming runner ?



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)
 public class SparkMetricsSinkTest {
+
+  @ClassRule
+  public static SparkSessionRule sessionRule =
+      new SparkSessionRule(
+          KV.of("spark.metrics.conf.*.sink.memory.class", InMemoryMetrics.class.getName()));
+
   @Rule public ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule();
 
+  @Rule
+  public TestPipeline pipeline = TestPipeline.fromOptions(sessionRule.createPipelineOptions());

Review Comment:
   IIRC runner, test mode etc are configured in `sessionRule#createPipelineOptions`



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)

Review Comment:
   I think it can be removed



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;

Review Comment:
   plus same in the other aggregatormetric class



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)
 public class SparkMetricsSinkTest {
+
+  @ClassRule
+  public static SparkSessionRule sessionRule =
+      new SparkSessionRule(
+          KV.of("spark.metrics.conf.*.sink.memory.class", InMemoryMetrics.class.getName()));
+
   @Rule public ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule();
 
+  @Rule
+  public TestPipeline pipeline = TestPipeline.fromOptions(sessionRule.createPipelineOptions());

Review Comment:
   final



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkBeamMetricTest.java:
##########
@@ -27,6 +27,7 @@
 
 /** Test SparkBeamMetric. */
 public class SparkBeamMetricTest {

Review Comment:
   Also I'm not sure specifying the JUnit 4 runner in BeamMetricTest is needed



-- 
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] echauchot commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   Run Spark ValidatesRunner


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;
+      }
+      try {
+        Gauge<Double> gauge = staticGauge(rawValue);
+        if (filter.matches(name, gauge)) {
+          metrics.put(name, gauge);
+        }
+      } catch (NumberFormatException e) {
+        LOG.warn(
+            "Metric `{}` of type {} can't be reported, conversion to double failed.",
+            name,
+            rawValue.getClass().getSimpleName(),
+            e);
+      }
+    }
+    return metrics;
+  }
+
+  // Metric type is assumed to be compatible with Double

Review Comment:
   Yes 👍 Among the know supported subtypes of `Metric` only `Gauge` and `MetricRegistry` fit. `MetricRegistry` would be even better, but also require a lot more changes. Spark 3 has a plugin framework which would simplify a lot of this (no custom sinks needed anymore + configuration nightmare for users). When migrating to a metrics plugin, this needs to turn into a registry. One thing at a 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 a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {
+
+  @Override
+  public final Map<String, Gauge<Double>> getValue() {
+    return getValue("", MetricFilter.ALL);
+  }
+
+  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
+
+  protected Gauge<Double> staticGauge(Number number) {
+    return new StaticGauge(number.doubleValue());
+  }
+
+  private static class StaticGauge implements Gauge<Double> {
+    double value;

Review Comment:
   @mosche make final for clarity



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);
   }
 
   public SparkSessionRule(String sparkMaster, KV<String, String>... sparkConfig) {
     this(sparkMaster, Arrays.stream(sparkConfig).collect(toMap(KV::getKey, KV::getValue)));
   }
 
+  private static int numDriverCores(String master) {

Review Comment:
   :+1 on the factory, I just looked at it



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;
+      }
+      try {
+        Gauge<Double> gauge = staticGauge(rawValue);
+        if (filter.matches(name, gauge)) {
+          metrics.put(name, gauge);
+        }
+      } catch (NumberFormatException e) {
+        LOG.warn(
+            "Metric `{}` of type {} can't be reported, conversion to double failed.",
+            name,
+            rawValue.getClass().getSimpleName(),
+            e);
+      }
+    }
+    return metrics;
+  }
+
+  // Metric type is assumed to be compatible with Double

Review Comment:
   👍  https://github.com/apache/beam/issues/22384



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java:
##########
@@ -21,14 +21,69 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-public class CodahaleGraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class CodahaleGraphiteSink implements Sink {

Review Comment:
   silently, with not even a ClassNotFoundException ?



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?

Review Comment:
   :+1:



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,33 +17,48 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import java.lang.reflect.Constructor;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag;
-import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
+  // default constructor to reflectively create static invoke expressions
+  private static final Constructor<StaticInvoke> STATIC_INVOKE_CONSTRUCTOR =
+      (Constructor<StaticInvoke>) StaticInvoke.class.getConstructors()[0];
+
+  static <T> ExpressionEncoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    return new ExpressionEncoder<>(serializer, deserializer, ClassTag.apply(clazz));
+  }
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    Expression serializer =
-        new EncoderHelpers.EncodeUsingBeamCoder<>(
-            new BoundReference(0, new ObjectType(clazz), true), coder);
-    Expression deserializer =
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(
-                new GetColumnByOrdinal(0, BinaryType), BinaryType, scala.Option.<String>empty()),
-            classTag,
-            coder);
-    return new ExpressionEncoder<>(serializer, deserializer, classTag);
+  /**
+   * Invoke method {@code fun} on Class {@code cls}, immediately propagating {@code null} if any
+   * input arg is {@code null}.
+   *
+   * <p>To address breaking interfaces between various version of Spark 3 these are created

Review Comment:
   I meant extract the spark version at the beginning of the pipeline and pass only an int to the methods.



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);

Review Comment:
   I agree it is better to leave as it is and open a ticket to improve scala interrop later on. Please open this ticket, assign it to you and refer it in this PR.



-- 
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] echauchot commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   @mosche as a general advice, it is easier for review process if you do tiny commits that address one comment (we will squash anyway) that way I can track the addressed comments and resolve the conversations myself


-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);

Review Comment:
   was just mentioning this as it is the custom for beam users. But I'm not sure it will entails any trouble for them. And I agree with the arguments you gave I'd add the local[1] could also obfuscate serialization issues. Fair enough for local[2]



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);

Review Comment:
   https://github.com/apache/beam/issues/22382



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -76,9 +67,10 @@ public void testInBatchMode() throws Exception {
             .apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()))
             .apply(new WordCount.CountWords())
             .apply(MapElements.via(new WordCount.FormatAsTextFn()));
+
     PAssert.that(output).containsInAnyOrder(EXPECTED_COUNTS);
-    pipeline.run();
+    pipeline.run().waitUntilFinish();

Review Comment:
   `waitUntilFish` is still the default with `testMode` on you mean ? So if testMode is activated why need to explicitely call `waitUntilFinish()` here ?



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.structuredstreaming.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {

Review Comment:
   👍 



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -76,9 +67,10 @@ public void testInBatchMode() throws Exception {
             .apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()))
             .apply(new WordCount.CountWords())
             .apply(MapElements.via(new WordCount.FormatAsTextFn()));
+
     PAssert.that(output).containsInAnyOrder(EXPECTED_COUNTS);
-    pipeline.run();
+    pipeline.run().waitUntilFinish();

Review Comment:
   it's not needed, already 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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkBeamMetricTest.java:
##########
@@ -27,6 +27,7 @@
 
 /** Test SparkBeamMetric. */
 public class SparkBeamMetricTest {

Review Comment:
   ping ?



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {
+
+  @Override
+  public final Map<String, Gauge<Double>> getValue() {
+    return getValue("", MetricFilter.ALL);
+  }
+
+  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
+
+  protected Gauge<Double> staticGauge(Number number) {
+    return new StaticGauge(number.doubleValue());
+  }
+
+  private static class StaticGauge implements Gauge<Double> {

Review Comment:
   👍 



##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {

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] mosche commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java:
##########
@@ -146,10 +146,12 @@ public SparkStructuredStreamingPipelineResult run(final Pipeline pipeline) {
             });
     executorService.shutdown();
 
-    // TODO: Streaming.
+    Runnable onTerminalState =
+        options.getUseActiveSparkSession()
+            ? () -> {}
+            : () -> translationContext.getSparkSession().stop();

Review Comment:
   makes sense, fair enough



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   @echauchot any comments / feedback already? we can also take this in turns, so you don't have to review everything at once


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();

Review Comment:
   If you have a look at `WithMetricsSupport` you can see that everything was previously returned as `Gauge` as well. Nothing has changed in that regard, this part was just a bit of a code cleanup on the way. For distributions the individual values are extracted as gauge each exactly the same was as before.
   



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/3/build.gradle:
##########
@@ -28,3 +28,37 @@ project.ext {
 
 // Load the main build script which contains all build logic.
 apply from: "$basePath/spark_runner.gradle"
+
+
+def sparkVersions = [
+    "330": "3.3.0",
+    "321": "3.2.1"
+]
+
+sparkVersions.each { kv ->

Review Comment:
   So, the runner is built against 3.1.2 but can run on 3.2 and 3.3 ? This is why you need to dynamically get the spark version either by reflexion or by getting it from the spark session. 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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/sink/CsvSink.java:
##########
@@ -18,22 +18,64 @@
 package org.apache.beam.runners.spark.metrics.sink;
 
 import com.codahale.metrics.MetricRegistry;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.lang.reflect.Constructor;
 import java.util.Properties;
 import org.apache.beam.runners.spark.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.metrics.WithMetricsSupport;
 import org.apache.spark.metrics.sink.Sink;
 
 /**
- * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to a CSV file.
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.metrics.sink.CsvSink"
+ * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
+ * "spark.metrics.conf.*.sink.csv.period"=10
+ * "spark.metrics.conf.*.sink.csv.unit"=seconds
+ * }</pre>
  */
-// Intentionally overriding parent name because inheritors should replace the parent.
-@SuppressFBWarnings("NM_SAME_SIMPLE_NAME_AS_SUPERCLASS")
-public class CsvSink extends org.apache.spark.metrics.sink.CsvSink {
+public class CsvSink implements Sink {
+
+  // Initialized reflectively as done by Spark's MetricsSystem
+  private final org.apache.spark.metrics.sink.CsvSink delegate;
+
+  /** Constructor for Spark 3.1.x. */
   public CsvSink(
       final Properties properties,
       final MetricRegistry metricRegistry,
       final org.apache.spark.SecurityManager securityMgr) {
-    super(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+  }
+
+  /** Constructor for Spark 3.2.x and later. */
+  public CsvSink(final Properties properties, final MetricRegistry metricRegistry) {
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry));
+  }
+
+  @Override
+  public void start() {
+    delegate.start();
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+
+  @Override
+  public void report() {
+    delegate.report();
+  }
+
+  private static org.apache.spark.metrics.sink.CsvSink newDelegate(Object... params) {
+    try {
+      Constructor<?> constructor = org.apache.spark.metrics.sink.CsvSink.class.getConstructors()[0];

Review Comment:
   :+1:  but it is still more readable I believe



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.
+ */
+public abstract class BeamMetricSet implements Gauge<Map<String, Gauge<Double>>> {
+
+  @Override
+  public final Map<String, Gauge<Double>> getValue() {
+    return getValue("", MetricFilter.ALL);
+  }
+
+  protected abstract Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter);
+
+  protected Gauge<Double> staticGauge(Number number) {
+    return new StaticGauge(number.doubleValue());
+  }
+
+  private static class StaticGauge implements Gauge<Double> {

Review Comment:
   ah I misunderstood the intent of the name. ConstantGauge lokks fine to me



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleCsvSink.java:
##########
@@ -21,16 +21,66 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
 /**
- * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to a CSV file.
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleCsvSink"
+ * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
+ * "spark.metrics.conf.*.sink.csv.period"=10
+ * "spark.metrics.conf.*.sink.csv.unit"=seconds
+ * }</pre>
  */
-public class CodahaleCsvSink extends org.apache.spark.metrics.sink.CsvSink {
+public class CodahaleCsvSink implements Sink {

Review Comment:
   let's keep this way to avoid silent breaking change



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java:
##########
@@ -35,13 +45,52 @@ public class EncoderHelpersTest {
 
   @ClassRule public static SparkSessionRule sessionRule = new SparkSessionRule();
 
+  private <T> Dataset<T> createDataset(List<T> data, Encoder<T> encoder) {
+    Dataset<T> ds = sessionRule.getSession().createDataset(data, encoder);
+    ds.printSchema();
+    return ds;
+  }
+
   @Test
   public void beamCoderToSparkEncoderTest() {
     List<Integer> data = Arrays.asList(1, 2, 3);
-    Dataset<Integer> dataset =
-        sessionRule
-            .getSession()
-            .createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of()));
+    Dataset<Integer> dataset = createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of()));
     assertEquals(data, dataset.collectAsList());
   }
+
+  @Test
+  public void testBeamEncoderOfPrivateType() {

Review Comment:
   ok



-- 
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] echauchot commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   > @echauchot any comments / feedback already? we can also take this in turns, so you don't have to review everything at once
   @mosche I was off on thrusday and friday so I have a dozen files left to review (only the metrics changes). I'll submit all the comments as a batch when I'm done with the review


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,33 +17,48 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import java.lang.reflect.Constructor;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag;
-import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
+  // default constructor to reflectively create static invoke expressions
+  private static final Constructor<StaticInvoke> STATIC_INVOKE_CONSTRUCTOR =
+      (Constructor<StaticInvoke>) StaticInvoke.class.getConstructors()[0];
+
+  static <T> ExpressionEncoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    return new ExpressionEncoder<>(serializer, deserializer, ClassTag.apply(clazz));
+  }
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    Expression serializer =
-        new EncoderHelpers.EncodeUsingBeamCoder<>(
-            new BoundReference(0, new ObjectType(clazz), true), coder);
-    Expression deserializer =
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(
-                new GetColumnByOrdinal(0, BinaryType), BinaryType, scala.Option.<String>empty()),
-            classTag,
-            coder);
-    return new ExpressionEncoder<>(serializer, deserializer, classTag);
+  /**
+   * Invoke method {@code fun} on Class {@code cls}, immediately propagating {@code null} if any
+   * input arg is {@code null}.
+   *
+   * <p>To address breaking interfaces between various version of Spark 3 these are created

Review Comment:
   I tried doing that initially and decided it's doing more harm than good it as it required passing the spark session allround, including static encoder factories and stuff like that.  I'll keep as is.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/3/build.gradle:
##########
@@ -28,3 +28,37 @@ project.ext {
 
 // Load the main build script which contains all build logic.
 apply from: "$basePath/spark_runner.gradle"
+
+
+def sparkVersions = [
+    "330": "3.3.0",
+    "321": "3.2.1"
+]
+
+sparkVersions.each { kv ->

Review Comment:
   That hasn't changed, that is still Spark 3.1.2. Considering the breaking changes in Spark it's better to stick with that version and not pump it in Beam at all. If would be a breaking change on the Beam side doing that!



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?

Review Comment:
   :+1: and it was already the case with previous code ?



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?
+            Stream.of(stripSuffix(cleanPart(key.stepName()))),

Review Comment:
   wondering, what's the value of keeping the step variable?



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
.test-infra/jenkins/job_PreCommit_Java_Spark3_Versions.groovy:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+import PrecommitJobBuilder
+
+PrecommitJobBuilder builder = new PrecommitJobBuilder(
+    scope: this,
+    nameBase: 'Java_Spark3_Versions',
+    gradleTask: ':runners:spark:3:sparkVersionsTest',
+    gradleSwitches: [
+      '-PdisableSpotlessCheck=true'

Review Comment:
   agree, fair enough



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;
+      }
+      try {
+        Gauge<Double> gauge = staticGauge(rawValue);
+        if (filter.matches(name, gauge)) {
+          metrics.put(name, gauge);
+        }
+      } catch (NumberFormatException e) {
+        LOG.warn(
+            "Metric `{}` of type {} can't be reported, conversion to double failed.",
+            name,
+            rawValue.getClass().getSimpleName(),
+            e);
+      }
+    }
+    return metrics;
+  }
+
+  // Metric type is assumed to be compatible with Double

Review Comment:
   please open a ticket for later on this matter.



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   @echauchot I've pushed all changes, pls have another look.


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java:
##########
@@ -19,256 +19,53 @@
 
 import static org.apache.spark.sql.types.DataTypes.BinaryType;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import org.apache.spark.sql.catalyst.expressions.BoundReference;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.catalyst.expressions.NonSQLExpression;
-import org.apache.spark.sql.catalyst.expressions.UnaryExpression;
-import org.apache.spark.sql.catalyst.expressions.codegen.Block;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator;
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext;
-import org.apache.spark.sql.catalyst.expressions.codegen.ExprCode;
+import org.apache.spark.sql.catalyst.expressions.Literal;
 import org.apache.spark.sql.types.DataType;
 import org.apache.spark.sql.types.ObjectType;
-import org.checkerframework.checker.nullness.qual.Nullable;
-import scala.StringContext;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.checkerframework.checker.nullness.qual.NonNull;
 
-/** {@link Encoders} utility class. */
-@SuppressWarnings({
-  "nullness" // TODO(https://github.com/apache/beam/issues/20497)
-})
 public class EncoderHelpers {
+  private static final DataType OBJECT_TYPE = new ObjectType(Object.class);
+
   /**
    * Wrap a Beam coder into a Spark Encoder using Catalyst Expression Encoders (which uses java code
    * generation).
    */
   public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    return EncoderFactory.fromBeamCoder(coder);
+    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
+    // Class T could be private, therefore use OBJECT_TYPE to not risk an IllegalAccessError
+    return EncoderFactory.create(
+        beamSerializer(rootRef(OBJECT_TYPE, true), coder),
+        beamDeserializer(rootCol(BinaryType), coder),

Review Comment:
   absolutely not, it compiles to the same code (just invoking the static coder utils)



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/SparkBeamMetric.java:
##########
@@ -33,61 +37,71 @@
 import org.apache.beam.sdk.metrics.MetricResult;
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
-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.base.Strings;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Streams;
 
 /**
- * An adapter between the {@link MetricsContainerStepMap} and Codahale's {@link Metric} interface.
+ * An adapter between the {@link MetricsContainerStepMap} and the Dropwizard {@link Metric}
+ * interface.
  */
-public class SparkBeamMetric implements Metric {
+public class SparkBeamMetric extends BeamMetricSet {
+
   private static final String ILLEGAL_CHARACTERS = "[^A-Za-z0-9-]";
 
-  static Map<String, ?> renderAll(MetricResults metricResults) {
-    Map<String, Object> metrics = new HashMap<>();
-    MetricQueryResults metricQueryResults = metricResults.allMetrics();
-    for (MetricResult<Long> metricResult : metricQueryResults.getCounters()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted());
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    MetricResults metricResults =
+        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    MetricQueryResults allMetrics = metricResults.allMetrics();
+    for (MetricResult<Long> metricResult : allMetrics.getCounters()) {
+      putFiltered(metrics, filter, renderName(prefix, metricResult), metricResult.getAttempted());
     }
-    for (MetricResult<DistributionResult> metricResult : metricQueryResults.getDistributions()) {
+    for (MetricResult<DistributionResult> metricResult : allMetrics.getDistributions()) {
       DistributionResult result = metricResult.getAttempted();
-      metrics.put(renderName(metricResult) + ".count", result.getCount());
-      metrics.put(renderName(metricResult) + ".sum", result.getSum());
-      metrics.put(renderName(metricResult) + ".min", result.getMin());
-      metrics.put(renderName(metricResult) + ".max", result.getMax());
-      metrics.put(renderName(metricResult) + ".mean", result.getMean());
+      String baseName = renderName(prefix, metricResult);
+      putFiltered(metrics, filter, baseName + ".count", result.getCount());
+      putFiltered(metrics, filter, baseName + ".sum", result.getSum());
+      putFiltered(metrics, filter, baseName + ".min", result.getMin());
+      putFiltered(metrics, filter, baseName + ".max", result.getMax());
+      putFiltered(metrics, filter, baseName + ".mean", result.getMean());
     }
-    for (MetricResult<GaugeResult> metricResult : metricQueryResults.getGauges()) {
-      metrics.put(renderName(metricResult), metricResult.getAttempted().getValue());
+    for (MetricResult<GaugeResult> metricResult : allMetrics.getGauges()) {
+      putFiltered(
+          metrics,
+          filter,
+          renderName(prefix, metricResult),
+          metricResult.getAttempted().getValue());
     }
     return metrics;
   }
 
-  Map<String, ?> renderAll() {
-    MetricResults metricResults =
-        asAttemptedOnlyMetricResults(MetricsAccumulator.getInstance().value());
-    return renderAll(metricResults);
-  }
-
   @VisibleForTesting
-  static String renderName(MetricResult<?> metricResult) {
+  @SuppressWarnings("nullness") // ok to have nullable elements on stream
+  static String renderName(String prefix, MetricResult<?> metricResult) {
     MetricKey key = metricResult.getKey();
     MetricName name = key.metricName();
-    String step = key.stepName();
-
-    ArrayList<String> pieces = new ArrayList<>();
+    return Streams.concat(
+            Stream.of(prefix), // prefix is not cleaned, should it be?

Review Comment:
   :+1: and it cas already the case with previous code ?



-- 
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] echauchot commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   Run Spark ValidatesRunner


-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)

Review Comment:
   it is removed from everywhere except SparkMetricsSinkTest in the SS runner



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java:
##########
@@ -35,13 +45,52 @@ public class EncoderHelpersTest {
 
   @ClassRule public static SparkSessionRule sessionRule = new SparkSessionRule();
 
+  private <T> Dataset<T> createDataset(List<T> data, Encoder<T> encoder) {
+    Dataset<T> ds = sessionRule.getSession().createDataset(data, encoder);
+    ds.printSchema();
+    return ds;
+  }
+
   @Test
   public void beamCoderToSparkEncoderTest() {
     List<Integer> data = Arrays.asList(1, 2, 3);
-    Dataset<Integer> dataset =
-        sessionRule
-            .getSession()
-            .createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of()));
+    Dataset<Integer> dataset = createDataset(data, EncoderHelpers.fromBeamCoder(VarIntCoder.of()));
     assertEquals(data, dataset.collectAsList());
   }
+
+  @Test
+  public void testBeamEncoderOfPrivateType() {

Review Comment:
   private types cannot be used in code generation, when trying to compile such code it would throw an `IllegalAccessError`.



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -76,9 +67,10 @@ public void testInBatchMode() throws Exception {
             .apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()))
             .apply(new WordCount.CountWords())
             .apply(MapElements.via(new WordCount.FormatAsTextFn()));
+
     PAssert.that(output).containsInAnyOrder(EXPECTED_COUNTS);
-    pipeline.run();
+    pipeline.run().waitUntilFinish();

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] mosche commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/3/build.gradle:
##########
@@ -28,3 +28,37 @@ project.ext {
 
 // Load the main build script which contains all build logic.
 apply from: "$basePath/spark_runner.gradle"
+
+
+def sparkVersions = [
+    "330": "3.3.0",
+    "321": "3.2.1"
+]
+
+sparkVersions.each { kv ->

Review Comment:
   That hasn't changed, that is still Spark 3.1.2. Considering the breaking changes in Spark it's better to stick with that version and not pump it in Beam at all. It would be a breaking change on the Beam side doing that.



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;

Review Comment:
   pushed already



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/AggregatorMetric.java:
##########
@@ -33,7 +41,35 @@ public static AggregatorMetric of(final NamedAggregators namedAggregators) {
     return new AggregatorMetric(namedAggregators);
   }
 
-  NamedAggregators getNamedAggregators() {
-    return namedAggregators;
+  @Override
+  public Map<String, Gauge<Double>> getValue(String prefix, MetricFilter filter) {
+    Map<String, Gauge<Double>> metrics = new HashMap<>();
+    for (Map.Entry<String, ?> entry : namedAggregators.renderAll().entrySet()) {
+      String name = prefix + "." + entry.getKey();
+      Object rawValue = entry.getValue();
+      if (rawValue == null) {
+        continue;

Review Comment:
   ping ?



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/BeamMetricSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.spark.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.MetricFilter;
+import java.util.Map;
+
+/**
+ * Map of Beam metrics available from {@link Gauge#getValue()}.
+ *
+ * <p>Note: Recent versions of Dropwizard {@link com.codahale.metrics.MetricRegistry MetricRegistry}
+ * do not allow registering arbitrary implementations of {@link com.codahale.metrics.Metric
+ * Metrics}.

Review Comment:
   ping ?



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)

Review Comment:
   jepp, i removed it



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)

Review Comment:
   jepp, i removed it already



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java:
##########
@@ -21,14 +21,69 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-public class CodahaleGraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class CodahaleGraphiteSink implements Sink {

Review Comment:
   That exception is caught by Spark and logged as error, but very easy to miss



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)

Review Comment:
   ok, thx, fixed that 👍 



-- 
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] echauchot commented on pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   Run Spark StructuredStreaming ValidatesRunner


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)
 public class SparkMetricsSinkTest {
+
+  @ClassRule
+  public static SparkSessionRule sessionRule =
+      new SparkSessionRule(
+          KV.of("spark.metrics.conf.*.sink.memory.class", InMemoryMetrics.class.getName()));
+
   @Rule public ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule();
 
+  @Rule
+  public TestPipeline pipeline = TestPipeline.fromOptions(sessionRule.createPipelineOptions());

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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -76,9 +67,10 @@ public void testInBatchMode() throws Exception {
             .apply(Create.of(WORDS).withCoder(StringUtf8Coder.of()))
             .apply(new WordCount.CountWords())
             .apply(MapElements.via(new WordCount.FormatAsTextFn()));
+
     PAssert.that(output).containsInAnyOrder(EXPECTED_COUNTS);
-    pipeline.run();
+    pipeline.run().waitUntilFinish();

Review Comment:
   :+1: 



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   @echauchot tests are green


-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/sink/CsvSink.java:
##########
@@ -18,22 +18,64 @@
 package org.apache.beam.runners.spark.metrics.sink;
 
 import com.codahale.metrics.MetricRegistry;
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.lang.reflect.Constructor;
 import java.util.Properties;
 import org.apache.beam.runners.spark.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.metrics.WithMetricsSupport;
 import org.apache.spark.metrics.sink.Sink;
 
 /**
- * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to a CSV file.
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to a CSV file.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.csv.class"="org.apache.beam.runners.spark.metrics.sink.CsvSink"
+ * "spark.metrics.conf.*.sink.csv.directory"="<output_directory>"
+ * "spark.metrics.conf.*.sink.csv.period"=10
+ * "spark.metrics.conf.*.sink.csv.unit"=seconds
+ * }</pre>
  */
-// Intentionally overriding parent name because inheritors should replace the parent.
-@SuppressFBWarnings("NM_SAME_SIMPLE_NAME_AS_SUPERCLASS")
-public class CsvSink extends org.apache.spark.metrics.sink.CsvSink {
+public class CsvSink implements Sink {
+
+  // Initialized reflectively as done by Spark's MetricsSystem
+  private final org.apache.spark.metrics.sink.CsvSink delegate;
+
+  /** Constructor for Spark 3.1.x. */
   public CsvSink(
       final Properties properties,
       final MetricRegistry metricRegistry,
       final org.apache.spark.SecurityManager securityMgr) {
-    super(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry), securityMgr);
+  }
+
+  /** Constructor for Spark 3.2.x and later. */
+  public CsvSink(final Properties properties, final MetricRegistry metricRegistry) {
+    delegate = newDelegate(properties, WithMetricsSupport.forRegistry(metricRegistry));
+  }
+
+  @Override
+  public void start() {
+    delegate.start();
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+
+  @Override
+  public void report() {
+    delegate.report();
+  }
+
+  private static org.apache.spark.metrics.sink.CsvSink newDelegate(Object... params) {
+    try {
+      Constructor<?> constructor = org.apache.spark.metrics.sink.CsvSink.class.getConstructors()[0];

Review Comment:
   👍 I can change that if you prefer, but it doesn't provide any additional safety / checks. That's the version i used initially. These constructors are only ever called by Spark's `MetricsSystem` (using reflection btw), so having anything else other than the default constructor would be just pointless / dead code.
   Also, in both cases `newInstance` uses varargs and it will always fail at runtime without any compilation error.



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java:
##########
@@ -146,10 +146,12 @@ public SparkStructuredStreamingPipelineResult run(final Pipeline pipeline) {
             });
     executorService.shutdown();
 
-    // TODO: Streaming.
+    Runnable onTerminalState =
+        options.getUseActiveSparkSession()
+            ? () -> {}
+            : () -> translationContext.getSparkSession().stop();

Review Comment:
   That transition doesn't exist anymore, the only way to get here is through `offerNewState` which by the time of this check already replaced `state` with `newState` (which is a terminal state and never `RUNNING`).
   This was previously only used to expose `stop()` for testing purposes.



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/2/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,38 +17,35 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import java.util.Collections;
-import java.util.List;
-import org.apache.beam.sdk.coders.Coder;
 import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
-import scala.collection.JavaConversions;
-import scala.reflect.ClassTag;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.Seq;
+import scala.collection.immutable.List;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    List<Expression> serializers =
-        Collections.singletonList(
-            new EncoderHelpers.EncodeUsingBeamCoder<>(
-                new BoundReference(0, new ObjectType(clazz), true), coder));
-
+  static <T> Encoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    List<Expression> serializers = Nil$.MODULE$.$colon$colon(serializer);
     return new ExpressionEncoder<>(
         SchemaHelpers.binarySchema(),
         false,
-        JavaConversions.collectionAsScalaIterable(serializers).toSeq(),
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(new GetColumnByOrdinal(0, BinaryType), BinaryType), classTag, coder),
-        classTag);
+        serializers,
+        deserializer,
+        ClassTag$.MODULE$.apply(clazz));

Review Comment:
   arf, no luck



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)

Review Comment:
   what can be removed?



##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java:
##########
@@ -21,51 +21,42 @@
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions;
-import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner;
+import org.apache.beam.runners.spark.structuredstreaming.SparkSessionRule;
 import org.apache.beam.runners.spark.structuredstreaming.examples.WordCount;
-import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 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.ImmutableSet;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExternalResource;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/**
- * TODO: add testInStreamingMode() once streaming support will be implemented.
- *
- * <p>A test that verifies Beam metrics are reported to Spark's metrics sink in both batch and
- * streaming modes.
- */
-@Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
+/** A test that verifies Beam metrics are reported to Spark's metrics sink in batch mode. */
 @RunWith(JUnit4.class)

Review Comment:
   what can be removed? you mean the 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] mosche commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderFactory.java:
##########
@@ -17,33 +17,48 @@
  */
 package org.apache.beam.runners.spark.structuredstreaming.translation.helpers;
 
-import static org.apache.spark.sql.types.DataTypes.BinaryType;
-
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.spark.sql.Encoder;
-import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal;
+import java.lang.reflect.Constructor;
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
-import org.apache.spark.sql.catalyst.expressions.BoundReference;
-import org.apache.spark.sql.catalyst.expressions.Cast;
 import org.apache.spark.sql.catalyst.expressions.Expression;
-import org.apache.spark.sql.types.ObjectType;
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke;
+import org.apache.spark.sql.types.DataType;
+import scala.collection.immutable.Nil$;
+import scala.collection.mutable.WrappedArray;
 import scala.reflect.ClassTag;
-import scala.reflect.ClassTag$;
 
 public class EncoderFactory {
+  // default constructor to reflectively create static invoke expressions
+  private static final Constructor<StaticInvoke> STATIC_INVOKE_CONSTRUCTOR =
+      (Constructor<StaticInvoke>) StaticInvoke.class.getConstructors()[0];
+
+  static <T> ExpressionEncoder<T> create(
+      Expression serializer, Expression deserializer, Class<? super T> clazz) {
+    return new ExpressionEncoder<>(serializer, deserializer, ClassTag.apply(clazz));
+  }
 
-  public static <T> Encoder<T> fromBeamCoder(Coder<T> coder) {
-    Class<? super T> clazz = coder.getEncodedTypeDescriptor().getRawType();
-    ClassTag<T> classTag = ClassTag$.MODULE$.apply(clazz);
-    Expression serializer =
-        new EncoderHelpers.EncodeUsingBeamCoder<>(
-            new BoundReference(0, new ObjectType(clazz), true), coder);
-    Expression deserializer =
-        new EncoderHelpers.DecodeUsingBeamCoder<>(
-            new Cast(
-                new GetColumnByOrdinal(0, BinaryType), BinaryType, scala.Option.<String>empty()),
-            classTag,
-            coder);
-    return new ExpressionEncoder<>(serializer, deserializer, classTag);
+  /**
+   * Invoke method {@code fun} on Class {@code cls}, immediately propagating {@code null} if any
+   * input arg is {@code null}.
+   *
+   * <p>To address breaking interfaces between various version of Spark 3 these are created

Review Comment:
   Unfortunately the problem remains the same... the static factories would effectively depend on the spark session (if you pass an int retrieved from a spark session or the session itself doesn't make a big difference here)



-- 
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] echauchot commented on a diff in pull request #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/SparkSessionRule.java:
##########
@@ -37,23 +39,43 @@ public SparkSessionRule(String sparkMaster, Map<String, String> sparkConfig) {
     builder = SparkSession.builder();
     sparkConfig.forEach(builder::config);
     builder.master(sparkMaster);
+    builder.config("spark.sql.shuffle.partitions", numDriverCores(sparkMaster));
   }
 
   public SparkSessionRule(KV<String, String>... sparkConfig) {
-    this("local", sparkConfig);
+    this("local[2]", sparkConfig);
   }
 
   public SparkSessionRule(String sparkMaster, KV<String, String>... sparkConfig) {
     this(sparkMaster, Arrays.stream(sparkConfig).collect(toMap(KV::getKey, KV::getValue)));
   }
 
+  private static int numDriverCores(String master) {

Review Comment:
   :+1:  on the factory, I just looked at it



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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


##########
runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/sink/CodahaleGraphiteSink.java:
##########
@@ -21,14 +21,69 @@
 import java.util.Properties;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.AggregatorMetric;
 import org.apache.beam.runners.spark.structuredstreaming.metrics.WithMetricsSupport;
+import org.apache.spark.SecurityManager;
 import org.apache.spark.metrics.sink.Sink;
 
-/** A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics to Graphite. */
-public class CodahaleGraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink {
+/**
+ * A {@link Sink} for <a href="https://spark.apache.org/docs/latest/monitoring.html#metrics">Spark's
+ * metric system</a> that is tailored to report {@link AggregatorMetric}s to Graphite.
+ *
+ * <p>The sink is configured using Spark configuration parameters, for example:
+ *
+ * <pre>{@code
+ * "spark.metrics.conf.*.sink.graphite.class"="org.apache.beam.runners.spark.structuredstreaming.metrics.sink.CodahaleGraphiteSink"
+ * "spark.metrics.conf.*.sink.graphite.host"="<graphite_hostname>"
+ * "spark.metrics.conf.*.sink.graphite.port"=<graphite_listening_port>
+ * "spark.metrics.conf.*.sink.graphite.period"=10
+ * "spark.metrics.conf.*.sink.graphite.unit"=seconds
+ * "spark.metrics.conf.*.sink.graphite.prefix"="<optional_prefix>"
+ * "spark.metrics.conf.*.sink.graphite.regex"="<optional_regex_to_send_matching_metrics>"
+ * }</pre>
+ */
+public class CodahaleGraphiteSink implements Sink {

Review Comment:
   I don't wanna introduce any breaking changes in this PR



-- 
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 #22157: Fixes #22156: Fix Spark3 runner to compile against Spark 3.2/3.3

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

   > as a general advice, it is easier for review process if you do tiny commits that address one comment (we will squash anyway) that way I can track the addressed comments and resolve the conversations myself
   
   fair comment @echauchot, but I guess we have to meet somewhere in the middle ... that would make addressing comments more than a hassle


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