You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "vsevolodstep-db (via GitHub)" <gi...@apache.org> on 2023/09/19 12:28:56 UTC

[GitHub] [spark] vsevolodstep-db opened a new pull request, #42995: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support

vsevolodstep-db opened a new pull request, #42995:
URL: https://github.com/apache/spark/pull/42995

   ### What changes were proposed in this pull request?
   This PR enhances existing ClosureCleaner implementation to support cleaning closures defined in Ammonite. Please refer to [this gist](https://gist.github.com/vsevolodstep-db/b8e4d676745d6e2d047ecac291e5254c) to get more context on how Ammonite code wrapping works and what problems I'm trying to solve here.
   
   As we need `ClosureCleaner` to be available in Spark Connect, I also moved the implementation to `common-utils` module. This brings a new `xbean-asm9-shaded` which is fairly small. 
   
   Existing implementation of `ClosureCleaner` also checks if the closure is serializable or not. This check is `spark-core` specific, so in order to preserve the existing code behaviour without changing other code pieces, I moved this check to `SparkClosureCleaner`, which is now used in `core`.
   
   The important changes affect `ClosureCleaner` only. They should not affect existing codepath for normal Scala closures / closures defined in a native Scala REPL and cover only closures defined in Ammonite.
   
   Also,  this PR modifies SparkConnect's `UserDefinedFunction` to actually use `ClosureCleaner`
   
   ### Why are the changes needed?
   To properly support closures defined in Ammonite, reduce UDF payload size and avoid possible `NonSerializable` exceptions. This includes:
   - lambda capturing outer command object, leading in a circular dependency
   - lambda capturing other command objects transitively, exploding payload size
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   Existing tests.
   New tests in `ReplE2ESuite` covering various scenarios using SparkConnect + Ammonite REPL to make sure closures are actually cleaned. 
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No


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


[GitHub] [spark] hvanhovell commented on a diff in pull request #42995: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support

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


##########
core/src/main/scala/org/apache/spark/util/Utils.scala:
##########
@@ -2313,7 +2234,7 @@ private[spark] object Utils
         e.getThrowables.asScala.exists(isBindCollision)
       case e: NativeIoException =>
         (e.getMessage != null && e.getMessage.startsWith("bind() failed: ")) ||
-          isBindCollision(e.getCause)
+        isBindCollision(e.getCause)

Review Comment:
   Spurious 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: 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


[GitHub] [spark] vsevolodstep-db commented on a diff in pull request #42995: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support

Posted by "vsevolodstep-db (via GitHub)" <gi...@apache.org>.
vsevolodstep-db commented on code in PR #42995:
URL: https://github.com/apache/spark/pull/42995#discussion_r1330221359


##########
common/utils/pom.xml:
##########
@@ -39,6 +39,10 @@
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-tags_${scala.binary.version}</artifactId>
     </dependency>
+    <dependency>

Review Comment:
   It's just a single jar containing shaded dependencies. It weights about 250KB



-- 
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-45136][CONNECT] Enhance ClosureCleaner with Ammonite support [spark]

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

   Merging to 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: 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-45136][CONNECT] Enhance ClosureCleaner with Ammonite support [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/util/ClosureCleaner.scala:
##########
@@ -46,11 +42,24 @@ private[spark] object ClosureCleaner extends Logging {
       null
     } else {
       val baos = new ByteArrayOutputStream(128)
-      Utils.copyStream(resourceStream, baos, true)
+
+      SparkStreamUtils.copyStream(resourceStream, baos, closeStreams = true)
       new ClassReader(new ByteArrayInputStream(baos.toByteArray))
     }
   }
 
+  private[util] def isAmmoniteCommandOrHelper(clazz: Class[_]): Boolean = clazz.getName.matches(
+    "^ammonite\\.\\$sess\\.cmd[0-9]*(\\$Helper\\$?)?")
+
+  private[util] def isDefinedInAmmonite(clazz: Class[_]): Boolean = clazz.getName.matches(
+    "^ammonite\\.\\$sess\\.cmd[0-9]*.*")

Review Comment:
   Ditto



##########
common/utils/src/main/scala/org/apache/spark/util/ClosureCleaner.scala:
##########
@@ -893,10 +1101,12 @@ private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM
           op: Int, owner: String, name: String, desc: String, itf: Boolean): Unit = {
         val argTypes = Type.getArgumentTypes(desc)
         if (op == INVOKESPECIAL && name == "<init>" && argTypes.length > 0
-            && argTypes(0).toString.startsWith("L") // is it an object?
-            && argTypes(0).getInternalName == myName) {
-          output += Utils.classForName(owner.replace('/', '.'),
-            initialize = false, noSparkClassLoader = true)
+          && argTypes(0).toString.startsWith("L") // is it an object?
+          && argTypes(0).getInternalName == myName) {

Review Comment:
   Is this format change necessary? I like the original better...
   (If this is from scalafmt then I have no problem with it 🤷 )



##########
common/utils/src/main/scala/org/apache/spark/util/ClosureCleaner.scala:
##########
@@ -46,11 +42,24 @@ private[spark] object ClosureCleaner extends Logging {
       null
     } else {
       val baos = new ByteArrayOutputStream(128)
-      Utils.copyStream(resourceStream, baos, true)
+
+      SparkStreamUtils.copyStream(resourceStream, baos, closeStreams = true)
       new ClassReader(new ByteArrayInputStream(baos.toByteArray))
     }
   }
 
+  private[util] def isAmmoniteCommandOrHelper(clazz: Class[_]): Boolean = clazz.getName.matches(
+    "^ammonite\\.\\$sess\\.cmd[0-9]*(\\$Helper\\$?)?")

Review Comment:
   Can we use Scala's triple quote string literal here to avoid the need for the escaping?
   i.e.
   ```scala
   """^ammonite\.\$sess\.cmd[0-9]*(\$Helper\$?)?"""
   ```



##########
common/utils/src/main/scala/org/apache/spark/util/ClosureCleaner.scala:
##########
@@ -893,10 +1101,12 @@ private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM
           op: Int, owner: String, name: String, desc: String, itf: Boolean): Unit = {
         val argTypes = Type.getArgumentTypes(desc)
         if (op == INVOKESPECIAL && name == "<init>" && argTypes.length > 0
-            && argTypes(0).toString.startsWith("L") // is it an object?
-            && argTypes(0).getInternalName == myName) {
-          output += Utils.classForName(owner.replace('/', '.'),
-            initialize = false, noSparkClassLoader = true)
+          && argTypes(0).toString.startsWith("L") // is it an object?
+          && argTypes(0).getInternalName == myName) {

Review Comment:
   Is this format change necessary? I like the original better...
   (If this is from scalafmt then I have no problem with it 🤷 )



##########
common/utils/src/main/scala/org/apache/spark/util/ClosureCleaner.scala:
##########
@@ -678,18 +851,45 @@ private[spark] object IndylambdaScalaClosures extends Logging {
     // Depth-first search for inner closures and track the fields that were accessed in them.
     // Start from the lambda body's implementation method, follow method invocations
     val visited = Set.empty[MethodIdentifier[_]]
-    val stack = Stack[MethodIdentifier[_]](implMethodId)
+    val queue = Queue[MethodIdentifier[_]](implMethodId)

Review Comment:
   Is this intended? What's the rationale for changing from stack => queue? Is breadth-first search preferred 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: 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-45136][CONNECT] Enhance ClosureCleaner with Ammonite support [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell closed pull request #42995: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support
URL: https://github.com/apache/spark/pull/42995


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


[GitHub] [spark] hvanhovell commented on a diff in pull request #42995: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support

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


##########
common/utils/pom.xml:
##########
@@ -39,6 +39,10 @@
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-tags_${scala.binary.version}</artifactId>
     </dependency>
+    <dependency>

Review Comment:
   How many transitive dependencies does this pull in? How large are they?



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


[GitHub] [spark] hvanhovell commented on a diff in pull request #42995: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support

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


##########
common/utils/src/main/scala/org/apache/spark/util/SparkStreamUtils.scala:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.util
+
+import java.io.{FileInputStream, FileOutputStream, InputStream, OutputStream}
+import java.nio.channels.{FileChannel, WritableByteChannel}
+
+import org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally
+
+private[spark] trait SparkStreamUtils {
+
+  /**
+   * Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream
+   * copying is disabled by default unless explicitly set transferToEnabled as true, the parameter
+   * transferToEnabled should be configured by spark.file.transferTo = [true|false].
+   */
+  def copyStream(
+      in: InputStream,
+      out: OutputStream,
+      closeStreams: Boolean = false,
+      transferToEnabled: Boolean = false): Long = {
+    tryWithSafeFinally {
+      (in, out) match {
+        case (input: FileInputStream, output: FileOutputStream) if transferToEnabled =>
+          // When both streams are File stream, use transferTo to improve copy performance.
+          val inChannel = input.getChannel
+          val outChannel = output.getChannel
+          val size = inChannel.size()
+          copyFileStreamNIO(inChannel, outChannel, 0, size)
+          size
+        case (input, output) =>
+          var count = 0L
+          val buf = new Array[Byte](8192)
+          var n = 0
+          while (n != -1) {
+            n = input.read(buf)
+            if (n != -1) {
+              output.write(buf, 0, n)
+              count += n
+            }
+          }
+          count
+      }
+    } {
+      if (closeStreams) {
+        try {
+          in.close()
+        } finally {
+          out.close()
+        }
+      }
+    }
+  }
+
+  def copyFileStreamNIO(
+      input: FileChannel,
+      output: WritableByteChannel,
+      startPosition: Long,
+      bytesToCopy: Long): Unit = {
+    val outputInitialState = output match {
+      case outputFileChannel: FileChannel =>
+        Some((outputFileChannel.position(), outputFileChannel))
+      case _ => None
+    }
+    var count = 0L
+    // In case transferTo method transferred less data than we have required.
+    while (count < bytesToCopy) {
+      count += input.transferTo(count + startPosition, bytesToCopy - count, output)
+    }
+    assert(
+      count == bytesToCopy,
+      s"request to copy $bytesToCopy bytes, but actually copied $count bytes.")
+
+    // Check the position after transferTo loop to see if it is in the right position and
+    // give user information if not.
+    // Position will not be increased to the expected length after calling transferTo in
+    // kernel version 2.6.32, this issue can be seen in
+    // https://bugs.openjdk.java.net/browse/JDK-7052359
+    // This will lead to stream corruption issue when using sort-based shuffle (SPARK-3948).
+    outputInitialState.foreach { case (initialPos, outputFileChannel) =>
+      val finalPos = outputFileChannel.position()
+      val expectedPos = initialPos + bytesToCopy
+      assert(
+        finalPos == expectedPos,
+        s"""
+           |Current position $finalPos do not equal to expected position $expectedPos
+           |after transferTo, please check your kernel version to see if it is 2.6.32,
+           |this is a kernel bug which will lead to unexpected behavior when using transferTo.
+           |You can set spark.file.transferTo = false to disable this NIO feature.
+         """.stripMargin)
+    }
+  }
+}
+
+object SparkStreamUtils extends SparkStreamUtils

Review Comment:
   `private[spark]`?



-- 
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-45136][CONNECT] Enhance ClosureCleaner with Ammonite support [spark]

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


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala:
##########
@@ -183,6 +183,7 @@ object ScalarUserDefinedFunction {
       function: AnyRef,
       inputEncoders: Seq[AgnosticEncoder[_]],
       outputEncoder: AgnosticEncoder[_]): ScalarUserDefinedFunction = {
+    SparkConnectClosureCleaner.clean(function)

Review Comment:
   So in theory we could defer serialization with your change. One of the motivations for doing it immediately that we would capture a partially constructed parent class; this prevents a self-reference which we can't deserialize. OTOH it is probably better to throw an exception where we construct the UDF, instead of throwing one when we submit the query for execution/analysis.



##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/application/ReplE2ESuite.scala:
##########
@@ -362,4 +362,103 @@ class ReplE2ESuite extends RemoteSparkSession with BeforeAndAfterEach {
     val output = runCommandsInShell(input)
     assertContains("noException: Boolean = true", output)
   }
+
+  test("closure cleaner") {
+    val input =
+      """
+        |class NonSerializable(val id: Int = -1) { }
+        |
+        |val x = 100; val y = new NonSerializable
+        |val t = 200
+        |def foo(): Int = { x }; def bar(): Int = { y.id }; val z = new NonSerializable

Review Comment:
   NIT: Is this a way to force ammonite to treat this as a single line (compilation unit)?
   
   If so, then using ```{ ... }``` is bit easier.



##########
core/src/main/scala/org/apache/spark/util/SparkClosureCleaner.scala:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.util
+
+import scala.collection.mutable
+
+import org.apache.spark.{SparkEnv, SparkException}
+
+private[spark] object SparkClosureCleaner {
+  /**
+   * Clean the given closure in place.
+   *
+   * More specifically, this renders the given closure serializable as long as it does not
+   * explicitly reference unserializable objects.
+   *
+   * @param closure           the closure to clean
+   * @param checkSerializable whether to verify that the closure is serializable after cleaning
+   * @param cleanTransitively whether to clean enclosing closures transitively
+   */
+  def clean(
+      closure: AnyRef,
+      checkSerializable: Boolean = true,
+      cleanTransitively: Boolean = true): Unit = {
+    if (ClosureCleaner.clean(closure, cleanTransitively, mutable.Map.empty)) {

Review Comment:
   NIT, for readability it is a nit nicer to put this in a boolean.



-- 
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-45136][CONNECT] Enhance ClosureCleaner with Ammonite support [spark]

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

   @vsevolodstep-db can you update the 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: 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-45136][CONNECT] Enhance ClosureCleaner with Ammonite support [spark]

Posted by "vsevolodstep-db (via GitHub)" <gi...@apache.org>.
vsevolodstep-db commented on code in PR #42995:
URL: https://github.com/apache/spark/pull/42995#discussion_r1345541797


##########
common/utils/src/main/scala/org/apache/spark/util/ClosureCleaner.scala:
##########
@@ -678,18 +851,45 @@ private[spark] object IndylambdaScalaClosures extends Logging {
     // Depth-first search for inner closures and track the fields that were accessed in them.
     // Start from the lambda body's implementation method, follow method invocations
     val visited = Set.empty[MethodIdentifier[_]]
-    val stack = Stack[MethodIdentifier[_]](implMethodId)
+    val queue = Queue[MethodIdentifier[_]](implMethodId)

Review Comment:
   Yes, this is intentional. Left a comment.
   To be able to get references to all visited ammonite command objects we need to make sure that "parent" (in reference tree)  object is processed before the child



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


[GitHub] [spark] hvanhovell commented on a diff in pull request #42995: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support

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


##########
common/utils/pom.xml:
##########
@@ -39,6 +39,10 @@
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-tags_${scala.binary.version}</artifactId>
     </dependency>
+    <dependency>

Review Comment:
   nice



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


[GitHub] [spark] hvanhovell commented on a diff in pull request #42995: [SPARK-45136][CONNECT] Enhance ClosureCleaner with Ammonite support

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


##########
core/src/main/scala/org/apache/spark/util/Utils.scala:
##########
@@ -93,11 +93,12 @@ private[spark] object CallSite {
  * Various utility methods used by Spark.
  */
 private[spark] object Utils
-  extends Logging
-  with SparkClassUtils
-  with SparkErrorUtils
-  with SparkFileUtils
-  with SparkSerDeUtils {
+    extends Logging

Review Comment:
   Is this the proper formatting?



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