You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "xupefei (via GitHub)" <gi...@apache.org> on 2024/03/25 14:08:21 UTC

[PR] [WIP] [SPARK-47545][Connect] Dataset `observe` support for the Scala client [spark]

xupefei opened a new pull request, #45701:
URL: https://github.com/apache/spark/pull/45701

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'common/utils/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1583392160


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala:
##########
@@ -198,6 +206,29 @@ private[sql] class SparkResult[T](
     nonEmpty
   }
 
+  private def processObservedMetrics(
+      metrics: java.util.List[ObservedMetrics]): Iterable[(String, Row)] = {
+    metrics.asScala.map { metric =>
+      assert(metric.getKeysCount == metric.getValuesCount)
+      var schema = new StructType()
+      val keys = mutable.ListBuffer.empty[String]
+      val values = mutable.ListBuffer.empty[Any]
+      (0 until metric.getKeysCount).map { i =>
+        val key = metric.getKeys(i)
+        val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i))
+        schema = schema.add(key, LiteralValueProtoConverter.toDataType(value.getClass))

Review Comment:
   There is a bit of a twist here. So, LiteralValueProtoConverter, returns a Tuple for a nested struct. This is not really expected in a Row. We can address this in a follow-up.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1557935463


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3397,7 +3488,11 @@ class Dataset[T] private[sql] (
     sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA)
   }
 
-  def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder)
+  def collectResult(): SparkResult[T] =
+    sparkSession.execute(plan, agnosticEncoder, getObservationsMapOpt)
+
+  def collectObservations(): Map[String, Map[String, Any]] =
+    collectResult().getObservedMetrics

Review Comment:
   Where are we using this?



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1558066618


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -363,6 +363,8 @@ object CheckConnectJvmClientCompatibility {
       ), // developer API
       ProblemFilters.exclude[DirectMissingMethodProblem](
         "org.apache.spark.sql.Dataset.collectResult"),
+      ProblemFilters.exclude[DirectMissingMethodProblem](

Review Comment:
   Can you also enable MiMa checks for observe by removing the problem filters for 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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1583350392


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,23 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    if (observationRegistry.containsKey(planId)) {

Review Comment:
   It is better to use putIfAbsent(...) here, if this returns a non-null value you can throw an exception.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1559249905


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -363,6 +363,8 @@ object CheckConnectJvmClientCompatibility {
       ), // developer API
       ProblemFilters.exclude[DirectMissingMethodProblem](
         "org.apache.spark.sql.Dataset.collectResult"),
+      ProblemFilters.exclude[DirectMissingMethodProblem](

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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1572146508


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,28 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    observation.synchronized {

Review Comment:
   Agreed. I changed it to a concurrent hash map.



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,28 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    observation.synchronized {

Review Comment:
   Agreed. I changed it to a concurrent hashmap.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1572263903


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,28 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    observation.synchronized {
+      if (observationRegistry.contains(planId)) {
+        throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+      }
+      observationRegistry.put(planId, observation)
+    }
+  }
+
+  private[sql] def setMetricsAndUnregisterObservation(
+      planId: Long,
+      metrics: Option[Map[String, Any]]): Unit = {
+    observationRegistry.get(planId).map { observation =>
+      if (observation.setMetricsAndNotify(metrics)) {
+        observationRegistry.remove(planId)

Review Comment:
   I looked at the code. It seems it's valid to check for non-empty metrics in Spark Core:
   ```scala
   private[spark] def onFinish(qe: QueryExecution): Unit = {
     ...
     val row: Option[Row] = qe.observedMetrics.get(name)
     val metrics: Option[Map[String, Any]] = row.map(r => r.getValuesMap[Any](r.schema.fieldNames.toImmutableArraySeq))
     if (setMetricsAndNotify(metrics)) {
       unregister()
     }
   }
   ```
   
   The option is for a case when the query finishes without metric. Is this possible?



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on PR #45701:
URL: https://github.com/apache/spark/pull/45701#issuecomment-2088626203

   > > @xupefei there is a genuine test failure. Can you check what is going on?
   > 
   > It seems the test is flaky, even after the previous attempt to fix it: #45173
   
   I re-ran the test and it did pass. The remaining failure is doc generation: `sh: 1: python: not found`.


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on PR #45701:
URL: https://github.com/apache/spark/pull/45701#issuecomment-2085537023

   @xupefei there is a genuine test failure. Can you check what is going on?


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1567173975


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala:
##########
@@ -27,18 +27,21 @@ import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch}
 import org.apache.arrow.vector.types.pojo
 
 import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics
+import org.apache.spark.sql.ObservationBase
 import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder}
 import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder}
 import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator}
-import org.apache.spark.sql.connect.common.DataTypeProtoConverter
+import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter}
 import org.apache.spark.sql.types.{DataType, StructType}
 import org.apache.spark.sql.util.ArrowUtils
 
 private[sql] class SparkResult[T](
     responses: CloseableIterator[proto.ExecutePlanResponse],
     allocator: BufferAllocator,
     encoder: AgnosticEncoder[T],
-    timeZoneId: String)
+    timeZoneId: String,
+    observationsOpt: Option[Map[Long, ObservationBase]] = None)

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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1566171078


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {
+
+  /**
+   * Create an Observation instance without providing a name. This generates a random name.
+   */
+  def this() = this(UUID.randomUUID().toString)
+
+  @volatile private var planId: Option[(SparkSession, Long)] = None
+
+  private[sql] def register(sparkSession: SparkSession, planId: Long): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    synchronized {
+      if (this.planId.isDefined) {
+        throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+      }
+      this.planId = Some((sparkSession, planId))
+    }
+
+    sparkSession.observationRegistry.put(planId, this)
+  }
+
+  private def unregister(): Unit = {
+    this.planId.map { case (sparkSession, planId) =>
+      sparkSession.observationRegistry.remove(planId)
+    }
+  }
+
+  override private[spark] def setMetricsAndNotify(metrics: Option[Map[String, Any]]): Boolean = {
+    this.unregister()
+    super.setMetricsAndNotify(metrics)
+  }
+}
+
+/**
+ * (Scala-specific) Create instances of Observation via Scala `apply`.
+ * @since 3.3.0

Review Comment:
   4.0.0



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1559266569


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3397,7 +3488,11 @@ class Dataset[T] private[sql] (
     sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA)
   }
 
-  def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder)
+  def collectResult(): SparkResult[T] =
+    sparkSession.execute(plan, agnosticEncoder, getObservationsMapOpt)
+
+  def collectObservations(): Map[String, Map[String, Any]] =
+    collectResult().getObservedMetrics

Review Comment:
   So far only in tests.
   Now this API is the only we we can fetch metrics that are not associated to `Observation` objects. It's serves the same propose as in the Python API, where we have `df.attrs["observed_metrics"]` to access such metrics.
   Please give suggestion about a better place to place it.



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3397,7 +3488,11 @@ class Dataset[T] private[sql] (
     sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA)
   }
 
-  def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder)
+  def collectResult(): SparkResult[T] =
+    sparkSession.execute(plan, agnosticEncoder, getObservationsMapOpt)
+
+  def collectObservations(): Map[String, Map[String, Any]] =
+    collectResult().getObservedMetrics

Review Comment:
   So far only in tests.
   Now this API is the only we we can fetch metrics that are not associated to `Observation` objects. It's serves the same propose as in the Python API, where we have `df.attrs["observed_metrics"]` to access such metrics.
   If possible, please give suggestion about a better place to place 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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1557934162


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -131,13 +131,25 @@ import org.apache.spark.util.SparkClassUtils
 class Dataset[T] private[sql] (
     val sparkSession: SparkSession,
     @DeveloperApi val plan: proto.Plan,
-    val encoder: Encoder[T])
+    val encoder: Encoder[T],
+    @DeveloperApi carryOverObservationsOpt: Option[Map[String, Observation]] = None)

Review Comment:
   How well does this work with plugins? Now everyone working on the scala client needs to be aware that they need to carry over this map. I much prefer that we send back the plan_id of the CollectMetrics node in the `ObservedMetrics` node.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1570681872


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala:
##########
@@ -198,6 +206,29 @@ private[sql] class SparkResult[T](
     nonEmpty
   }
 
+  private def processObservedMetrics(
+      metrics: java.util.List[ObservedMetrics]): Iterable[(String, Row)] = {
+    metrics.asScala.map { metric =>
+      assert(metric.getKeysCount == metric.getValuesCount)
+      var schema = new StructType()
+      val keys = mutable.ListBuffer.empty[String]
+      val values = mutable.ListBuffer.empty[Any]
+      (0 until metric.getKeysCount).map { i =>
+        val key = metric.getKeys(i)
+        val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i))
+        schema = schema.add(key, LiteralValueProtoConverter.toDataType(value.getClass))

Review Comment:
   I didn't see `.dataType` for the literal `metric.getValues(i)`, only `.getLiteralTypeCase`.
   Also to get the actual DataType, we have to look at the values, for example for `CalendarInterval`. I think we can't avoid looking at the converted values anyway.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1566210772


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3397,7 +3488,11 @@ class Dataset[T] private[sql] (
     sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA)
   }
 
-  def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder)
+  def collectResult(): SparkResult[T] =
+    sparkSession.execute(plan, agnosticEncoder, getObservationsMapOpt)
+
+  def collectObservations(): Map[String, Map[String, Any]] =
+    collectResult().getObservedMetrics

Review Comment:
   If it is only tests then drop it, and use `collectResult().getObservedMetrics` instead.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "ueshin (via GitHub)" <gi...@apache.org>.
ueshin commented on PR #45701:
URL: https://github.com/apache/spark/pull/45701#issuecomment-2025839011

   So `df.collectObservations()` seems to be a new API available only in Spark Connect Scala client?


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1566170352


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {
+
+  /**
+   * Create an Observation instance without providing a name. This generates a random name.
+   */
+  def this() = this(UUID.randomUUID().toString)
+
+  @volatile private var planId: Option[(SparkSession, Long)] = None
+
+  private[sql] def register(sparkSession: SparkSession, planId: Long): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    synchronized {

Review Comment:
   Please replace this by a method in SparkSession called `private[sql] def registerObservation(planId: Long, observation: Observation): Unit`



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1566163349


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala:
##########
@@ -27,18 +27,21 @@ import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch}
 import org.apache.arrow.vector.types.pojo
 
 import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics
+import org.apache.spark.sql.ObservationBase
 import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder}
 import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder}
 import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator}
-import org.apache.spark.sql.connect.common.DataTypeProtoConverter
+import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter}
 import org.apache.spark.sql.types.{DataType, StructType}
 import org.apache.spark.sql.util.ArrowUtils
 
 private[sql] class SparkResult[T](
     responses: CloseableIterator[proto.ExecutePlanResponse],
     allocator: BufferAllocator,
     encoder: AgnosticEncoder[T],
-    timeZoneId: String)
+    timeZoneId: String,
+    observationsOpt: Option[Map[Long, ObservationBase]] = None)

Review Comment:
   Can you just pass in a callback? The result does not need to know that this is backed by a map.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on PR #45701:
URL: https://github.com/apache/spark/pull/45701#issuecomment-2088228423

   > @xupefei there is a genuine test failure. Can you check what is going on?
   
   It seems the test is flaky, even after the previous attempt to fix it: https://github.com/apache/spark/pull/45173


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1557936456


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {

Review Comment:
   Why do we need this subclass?



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on PR #45701:
URL: https://github.com/apache/spark/pull/45701#issuecomment-2031447755

   > So `df.collectObservations()` seems to be a new API available only in Spark Connect Scala client?
   
   Yes, similar to `df.attrs["observed_metrics"]` which is only in the Python client.


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1571122260


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala:
##########
@@ -27,18 +27,22 @@ import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch}
 import org.apache.arrow.vector.types.pojo
 
 import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics
+import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder}
 import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder}
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
 import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator}
-import org.apache.spark.sql.connect.common.DataTypeProtoConverter
+import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter}
 import org.apache.spark.sql.types.{DataType, StructType}
 import org.apache.spark.sql.util.ArrowUtils
 
 private[sql] class SparkResult[T](
     responses: CloseableIterator[proto.ExecutePlanResponse],
     allocator: BufferAllocator,
     encoder: AgnosticEncoder[T],
-    timeZoneId: String)
+    timeZoneId: String,
+    setObservationMetricsOpt: Option[(Long, Option[Map[String, Any]]) => Unit] = None)

Review Comment:
   Why does the callback need to handle `None` values? From looking at the code we will only invoke the callback when there is a meaningful result.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1571105638


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,28 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    observation.synchronized {
+      if (observationRegistry.contains(planId)) {
+        throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+      }
+      observationRegistry.put(planId, observation)
+    }
+  }
+
+  private[sql] def setMetricsAndUnregisterObservation(
+      planId: Long,
+      metrics: Option[Map[String, Any]]): Unit = {
+    observationRegistry.get(planId).map { observation =>
+      if (observation.setMetricsAndNotify(metrics)) {
+        observationRegistry.remove(planId)

Review Comment:
   Should this be tied to whether or not the observation has been successfully updated? Other question under what circumstance can the metrics be empty.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1567676904


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {
+
+  /**
+   * Create an Observation instance without providing a name. This generates a random name.
+   */
+  def this() = this(UUID.randomUUID().toString)
+
+  @volatile private var planId: Option[(SparkSession, Long)] = None
+
+  private[sql] def register(sparkSession: SparkSession, planId: Long): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    synchronized {
+      if (this.planId.isDefined) {
+        throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+      }
+      this.planId = Some((sparkSession, planId))
+    }
+
+    sparkSession.observationRegistry.put(planId, this)
+  }
+
+  private def unregister(): Unit = {

Review Comment:
   Thanks! I moved the register/unregister logic to `SparkSession`.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1559250767


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {

Review Comment:
   Oh true, this one is not needed. We could use ObservationBase directly.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1566169201


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {
+
+  /**
+   * Create an Observation instance without providing a name. This generates a random name.
+   */
+  def this() = this(UUID.randomUUID().toString)
+
+  @volatile private var planId: Option[(SparkSession, Long)] = None
+
+  private[sql] def register(sparkSession: SparkSession, planId: Long): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    synchronized {
+      if (this.planId.isDefined) {
+        throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+      }
+      this.planId = Some((sparkSession, planId))
+    }
+
+    sparkSession.observationRegistry.put(planId, this)
+  }
+
+  private def unregister(): Unit = {

Review Comment:
   How about we move this into the SparkSession? It is a bit weird that we are manipulating state here.
   
   I would just make that part of the invoke and remove logic we need to implement anyway.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1572264833


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,28 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    observation.synchronized {
+      if (observationRegistry.contains(planId)) {
+        throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+      }
+      observationRegistry.put(planId, observation)
+    }
+  }
+
+  private[sql] def setMetricsAndUnregisterObservation(
+      planId: Long,
+      metrics: Option[Map[String, Any]]): Unit = {
+    observationRegistry.get(planId).map { observation =>
+      if (observation.setMetricsAndNotify(metrics)) {
+        observationRegistry.remove(planId)

Review Comment:
   But nevertheless we don't need to handle this case in Connect because we look up using Plan Id.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell closed pull request #45701: [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client
URL: https://github.com/apache/spark/pull/45701


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1566146921


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {

Review Comment:
   Yes, we have to call it Observation. Naming it anything else will break compatibility. However my question is more about why we need a subclass? Is there 'frontend' specific code involved?



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1566164128


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala:
##########
@@ -79,6 +82,7 @@ private[sql] class SparkResult[T](
   private[this] var arrowSchema: pojo.Schema = _
   private[this] var nextResultIndex: Int = 0
   private val resultMap = mutable.Map.empty[Int, (Long, Seq[ArrowMessage])]
+  private val observedMetrics = mutable.Map.empty[String, Map[String, Any]]

Review Comment:
   Please make it Map[String, Row].



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [WIP] [SPARK-47545][Connect] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on PR #45701:
URL: https://github.com/apache/spark/pull/45701#issuecomment-2020741279

   > Let's make sure this is matched with Python verison.
   
   Thanks Hyukjin! Yes the behavior largely matches the Python version (internals and user-facing APIs). One difference though is the `df.collectObservations()` API which in Python has a different syntax `df.attrs["observed_metrics"]`.


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1571102181


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,28 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    observation.synchronized {

Review Comment:
   We should either lock the map, or use a concurrent hash map. Locking the observation should only be needed when we set the value, and even then the observation should be responsible for 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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1559262517


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {

Review Comment:
   The purpose of this `Observation` class is to provide a familiar name for users, avoiding the need to learn a new one. This is necessary because all of spark-core, spark-api, and spark-connect-client share the same class path, thus we must distinct the parent class `ObservationBase` from others.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1572265083


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala:
##########
@@ -27,18 +27,22 @@ import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch}
 import org.apache.arrow.vector.types.pojo
 
 import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics
+import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder}
 import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder}
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
 import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator}
-import org.apache.spark.sql.connect.common.DataTypeProtoConverter
+import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter}
 import org.apache.spark.sql.types.{DataType, StructType}
 import org.apache.spark.sql.util.ArrowUtils
 
 private[sql] class SparkResult[T](
     responses: CloseableIterator[proto.ExecutePlanResponse],
     allocator: BufferAllocator,
     encoder: AgnosticEncoder[T],
-    timeZoneId: String)
+    timeZoneId: String,
+    setObservationMetricsOpt: Option[(Long, Option[Map[String, Any]]) => Unit] = None)

Review Comment:
   True. 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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1583349498


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,23 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    if (observationRegistry.containsKey(planId)) {
+      throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+    }
+    observationRegistry.put(planId, observation)
+  }
+
+  private[sql] def setMetricsAndUnregisterObservation(
+      planId: Long,
+      metrics: Map[String, Any]): Unit = {
+    if (observationRegistry.containsKey(planId)) {

Review Comment:
   From a threading point of view i'd be better not to touch the map twice. Just try to remove the observation for the planId, and check if it is not null.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "ueshin (via GitHub)" <gi...@apache.org>.
ueshin commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1543431051


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3338,7 +3358,25 @@ class Dataset[T] private[sql] (
   }
 
   def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = {

Review Comment:
   Don't we need to annotate this with `@varargs` for Java client?



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3338,7 +3358,25 @@ class Dataset[T] private[sql] (
   }
 
   def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = {
-    throw new UnsupportedOperationException("observe is not implemented.")
+    sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder =>
+      builder.getCollectMetricsBuilder
+        .setInput(plan.getRoot)
+        .setName(name)
+        .addAllMetrics((expr +: exprs).map(_.expr).asJava)
+    }
+  }
+
+  def observe(observation: Observation, expr: Column, exprs: Column*): Dataset[T] = {

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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1547415263


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3338,7 +3358,25 @@ class Dataset[T] private[sql] (
   }
 
   def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = {

Review Comment:
   Good catch. Fixed!



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1559250767


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql
+
+import java.util.UUID
+
+class Observation(name: String) extends ObservationBase(name) {

Review Comment:
   Oh true, this one is not needed. We could use ObservationBase directly.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1561082565


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -131,13 +131,25 @@ import org.apache.spark.util.SparkClassUtils
 class Dataset[T] private[sql] (
     val sparkSession: SparkSession,
     @DeveloperApi val plan: proto.Plan,
-    val encoder: Encoder[T])
+    val encoder: Encoder[T],
+    @DeveloperApi carryOverObservationsOpt: Option[Map[String, Observation]] = None)

Review Comment:
   Refactor done in the latest comments. I took your suggestion and made a change to store a map of Plan-Id-to-Observation in SparkSession.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1567644036


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -3397,7 +3488,11 @@ class Dataset[T] private[sql] (
     sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA)
   }
 
-  def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder)
+  def collectResult(): SparkResult[T] =
+    sparkSession.execute(plan, agnosticEncoder, getObservationsMapOpt)
+
+  def collectObservations(): Map[String, Map[String, Any]] =
+    collectResult().getObservedMetrics

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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1569265956


##########
connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala:
##########
@@ -198,6 +206,29 @@ private[sql] class SparkResult[T](
     nonEmpty
   }
 
+  private def processObservedMetrics(
+      metrics: java.util.List[ObservedMetrics]): Iterable[(String, Row)] = {
+    metrics.asScala.map { metric =>
+      assert(metric.getKeysCount == metric.getValuesCount)
+      var schema = new StructType()
+      val keys = mutable.ListBuffer.empty[String]
+      val values = mutable.ListBuffer.empty[Any]
+      (0 until metric.getKeysCount).map { i =>
+        val key = metric.getKeys(i)
+        val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i))
+        schema = schema.add(key, LiteralValueProtoConverter.toDataType(value.getClass))

Review Comment:
   The unconverted literal should have a `dataType`. Can we use that instead?



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1572148735


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,28 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    observation.synchronized {
+      if (observationRegistry.contains(planId)) {
+        throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+      }
+      observationRegistry.put(planId, observation)
+    }
+  }
+
+  private[sql] def setMetricsAndUnregisterObservation(
+      planId: Long,
+      metrics: Option[Map[String, Any]]): Unit = {
+    observationRegistry.get(planId).map { observation =>
+      if (observation.setMetricsAndNotify(metrics)) {
+        observationRegistry.remove(planId)

Review Comment:
   I had the same question when I looked at the code. In Spark Core we only de-register the Observation when some non-empty metrics are set. I am not sure under which circumstance the metrics can be empty.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "xupefei (via GitHub)" <gi...@apache.org>.
xupefei commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1572148735


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala:
##########
@@ -813,6 +823,28 @@ class SparkSession private[sql] (
    * Set to false to prevent client.releaseSession on close() (testing only)
    */
   private[sql] var releaseSessionOnClose = true
+
+  private[sql] def registerObservation(planId: Long, observation: Observation): Unit = {
+    // makes this class thread-safe:
+    // only the first thread entering this block can set sparkSession
+    // all other threads will see the exception, as it is only allowed to do this once
+    observation.synchronized {
+      if (observationRegistry.contains(planId)) {
+        throw new IllegalArgumentException("An Observation can be used with a Dataset only once")
+      }
+      observationRegistry.put(planId, observation)
+    }
+  }
+
+  private[sql] def setMetricsAndUnregisterObservation(
+      planId: Long,
+      metrics: Option[Map[String, Any]]): Unit = {
+    observationRegistry.get(planId).map { observation =>
+      if (observation.setMetricsAndNotify(metrics)) {
+        observationRegistry.remove(planId)

Review Comment:
   I had the same question when I looked at the code. In Spark Core we only de-register the Observation when some non-empty metrics are set, so I decide to keep it the same in Connect. I am not sure under which circumstance the metrics can be empty.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on code in PR #45701:
URL: https://github.com/apache/spark/pull/45701#discussion_r1571108161


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala:
##########
@@ -1511,6 +1514,46 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
         (0 until 5).foreach(i => assert(row.get(i * 2) === row.get(i * 2 + 1)))
       }
   }
+
+  test("Observable metrics") {
+    val df = spark.range(99).withColumn("extra", col("id") - 1)
+    val ob1 = new Observation("ob1")
+    val observedDf = df.observe(ob1, min("id"), avg("id"), max("id"))
+    val observedObservedDf = observedDf.observe("ob2", min("extra"), avg("extra"), max("extra"))
+
+    val ob1Schema = new StructType()
+      .add("min(id)", LongType)
+      .add("avg(id)", DoubleType)
+      .add("max(id)", LongType)
+    val ob2Schema = new StructType()
+      .add("min(extra)", LongType)
+      .add("avg(extra)", DoubleType)
+      .add("max(extra)", LongType)
+    val ob1Metrics = Map("ob1" -> new GenericRowWithSchema(Array(0, 49, 98), ob1Schema))
+    val ob2Metrics = Map("ob2" -> new GenericRowWithSchema(Array(-1, 48, 97), ob2Schema))
+
+    assert(df.collectResult().getObservedMetrics === Map.empty)
+    assert(observedDf.collectResult().getObservedMetrics === ob1Metrics)
+    assert(observedObservedDf.collectResult().getObservedMetrics === ob1Metrics ++ ob2Metrics)
+  }
+
+  test("Observation.get is blocked until the query is finished") {
+    val df = spark.range(99).withColumn("extra", col("id") - 1)
+    val observation = new Observation("ob1")
+    val observedDf = df.observe(observation, min("id"), avg("id"), max("id"))
+
+    // Start a new thread to get the observation
+    val future = Future(observation.get)(ExecutionContext.global)

Review Comment:
   For the record. IMO the observation class should have been using a future from the get go.



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47545][CONNECT] Dataset `observe` support for the Scala client [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell commented on PR #45701:
URL: https://github.com/apache/spark/pull/45701#issuecomment-2101304531

   Merging!


-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org