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/10/12 19:02:44 UTC

[PR] [SPARK-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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

   ### What changes were proposed in this pull request?
   This PR extends existing SparkConnect `SparkSession.addArtifact` API to support Apache Ivy URIs in extend to the existing local `.jar` & `.class` file support. It leverages the existing support of Apache Ivy which was previously a part of spark-code (`SparkSubmitUtils`)
   
   This PR contains the following changes:
   1. Refactoring `SparkSubmitUtils` and moving Ivy-related parts to `commons` module. This results in adding two small dependencies to `common` (`apache ivy` and `commons-io`)
   2. Extracting some parts of `Utils` & `TestUtils` to `common`
   3. Extending SparkConnect `SparkSession.addArtifact` to support Ivy URIs
   
   Refactoring changes done in 1-2 are not altering code functionality. These changes are mostly about moving code between files & packages and adjusting some dependencies
   
   ### Why are the changes needed?
   It's possible to add maven artifacts to Spark, but SparkConnect currently lacks this functionality
   
   ### Does this PR introduce _any_ user-facing change?
   Yes: it extends the existing SparkConnect `SparkSession.addArtifact` API to support more URI types
   
   ### How was this patch tested?
   Existing tests covering Ivy resolution functionality;
   New test in `ArtifactSuite` testing ivy support in `ArtifactManager`
   New E2E test in `ReplE2ESuite` testing adding maven library and using it from UDF
   
   ### 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


Re: [PR] [SPARK-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/pom.xml:
##########
@@ -51,6 +51,20 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-text</artifactId>
     </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.ivy</groupId>
+      <artifactId>ivy</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>oro</groupId>

Review Comment:
   BTW oro is pretty much dead. Why does ivy need 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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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

   @HyukjinKwon are the current python failures a known problem?


-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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

   Merged 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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/util/IvyTestUtils.scala:
##########
@@ -15,30 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.{File, FileInputStream, FileOutputStream}
-import java.util.jar.{JarEntry, JarOutputStream}
+import java.util.jar.{JarEntry, JarOutputStream, Manifest}
 import java.util.jar.Attributes.Name
-import java.util.jar.Manifest
 
 import scala.collection.mutable.ArrayBuffer
 
-import com.google.common.io.{ByteStreams, Files}
 import org.apache.commons.io.FileUtils
 import org.apache.ivy.core.settings.IvySettings
 
-import org.apache.spark.TestUtils.{createCompiledClass, JavaSourceFromString}
-import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate
-import org.apache.spark.util.Utils
+import org.apache.spark.util.MavenUtils.MavenCoordinate
 
-private[deploy] object IvyTestUtils {
+private[spark] object IvyTestUtils {

Review Comment:
   +1



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

To unsubscribe, e-mail: 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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/pom.xml:
##########
@@ -51,6 +51,20 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-text</artifactId>
     </dependency>
+    <dependency>

Review Comment:
   ~400 Kb from commons-io and ~1Mb from Ivy 



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/util/IvyTestUtils.scala:
##########
@@ -330,58 +341,78 @@ private[deploy] object IvyTestUtils {
 
   /**
    * Creates a suite of jars and poms, with or without dependencies, mocking a maven repository.
-   * @param artifact The main maven coordinate to generate the jar and pom for.
-   * @param dependencies List of dependencies this artifact might have to also create jars and poms.
-   * @param rootDir The root folder of the repository (like `~/.m2/repositories`)
-   * @param useIvyLayout whether to mock the Ivy layout for local repository testing
-   * @param withPython Whether to pack python files inside the jar for extensive testing.
-   * @return Root path of the repository. Will be `rootDir` if supplied.
+   * @param artifact
+   *   The main maven coordinate to generate the jar and pom for.
+   * @param dependencies
+   *   List of dependencies this artifact might have to also create jars and poms.
+   * @param rootDir
+   *   The root folder of the repository (like `~/.m2/repositories`)
+   * @param useIvyLayout
+   *   whether to mock the Ivy layout for local repository testing
+   * @param withPython
+   *   Whether to pack python files inside the jar for extensive testing.
+   * @return
+   *   Root path of the repository. Will be `rootDir` if supplied.
    */
-  private[deploy] def createLocalRepositoryForTests(
+  private[spark] def createLocalRepositoryForTests(
       artifact: MavenCoordinate,
       dependencies: Option[String],
       rootDir: Option[File],
       useIvyLayout: Boolean = false,
       withPython: Boolean = false,
       withR: Boolean = false): File = {
-    val deps = dependencies.map(SparkSubmitUtils.extractMavenCoordinates)
+    val deps = dependencies.map(MavenUtils.extractMavenCoordinates)
     val mainRepo = createLocalRepository(artifact, deps, rootDir, useIvyLayout, withPython, withR)
-    deps.foreach { seq => seq.foreach { dep =>
-      createLocalRepository(dep, None, Some(mainRepo), useIvyLayout, withPython = false)
-    }}
+    deps.foreach { seq =>
+      seq.foreach { dep =>
+        createLocalRepository(dep, None, Some(mainRepo), useIvyLayout, withPython = false)
+      }
+    }
     mainRepo
   }
 
   /**
    * Creates a repository for a test, and cleans it up afterwards.
    *
-   * @param artifact The main maven coordinate to generate the jar and pom for.
-   * @param dependencies List of dependencies this artifact might have to also create jars and poms.
-   * @param rootDir The root folder of the repository (like `~/.m2/repositories`)
-   * @param useIvyLayout whether to mock the Ivy layout for local repository testing
-   * @param withPython Whether to pack python files inside the jar for extensive testing.
-   * @return Root path of the repository. Will be `rootDir` if supplied.
+   * @param artifact
+   *   The main maven coordinate to generate the jar and pom for.
+   * @param dependencies
+   *   List of dependencies this artifact might have to also create jars and poms.
+   * @param rootDir
+   *   The root folder of the repository (like `~/.m2/repositories`)
+   * @param useIvyLayout
+   *   whether to mock the Ivy layout for local repository testing
+   * @param withPython
+   *   Whether to pack python files inside the jar for extensive testing.
+   * @return
+   *   Root path of the repository. Will be `rootDir` if supplied.
    */
-  private[deploy] def withRepository(
+  private[spark] def withRepository(
       artifact: MavenCoordinate,
       dependencies: Option[String],
       rootDir: Option[File],
       useIvyLayout: Boolean = false,
       withPython: Boolean = false,
       withR: Boolean = false,
       ivySettings: IvySettings = new IvySettings)(f: String => Unit): Unit = {
-    val deps = dependencies.map(SparkSubmitUtils.extractMavenCoordinates)
+    val deps = dependencies.map(MavenUtils.extractMavenCoordinates)
     purgeLocalIvyCache(artifact, deps, ivySettings)
-    val repo = createLocalRepositoryForTests(artifact, dependencies, rootDir, useIvyLayout,

Review Comment:
   Sure, reverted scalafmt changes not related to the actual PR changes



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

To unsubscribe, e-mail: 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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/test/scala/org/apache/spark/util/MavenUtilsSuite.scala:
##########
@@ -94,19 +97,20 @@ class SparkSubmitUtilsSuite extends SparkFunSuite {
   }
 
   test("add dependencies works correctly") {
-    val md = SparkSubmitUtils.getModuleDescriptor
-    val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.12:0.1," +
-      "com.databricks:spark-avro_2.12:0.1")
+    val md = MavenUtils.getModuleDescriptor
+    val artifacts = MavenUtils.extractMavenCoordinates(
+      "com.databricks:spark-csv_2.12:0.1," +
+        "com.databricks:spark-avro_2.12:0.1")

Review Comment:
   e.g., this too



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ArtifactSuite.scala:
##########
@@ -268,4 +269,14 @@ class ArtifactSuite extends ConnectFunSuite with BeforeAndAfterEach {
     val receivedRequests = service.getAndClearLatestAddArtifactRequests()
     assert(receivedRequests.size == 1)
   }
+
+  test("resolve ivy") {
+    val artifacts =
+      Artifact.newIvyArtifacts(URI.create("ivy://org.apache.hive:hive-storage-api:2.7.0"))

Review Comment:
   Probably, but many tests in `SparkContextSuite` also do this, though this is not a strong supporting argument 
   
   I'm wondering if we should add `?repos=...`  parameter (or something similar) to the Ivy URI to support providing additional private / local repositories? Spark-core does support this by using Spark confs, but we can't do this in connect.
   
   This will add some useful functionality and will allow us to test this without relying on remote repositories
   
   



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon closed pull request #43354: [SPARK-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact
URL: https://github.com/apache/spark/pull/43354


-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2452,7 +2452,7 @@ package object config {
       .createOptional
 
   private[spark] val JAR_IVY_SETTING_PATH =
-    ConfigBuilder("spark.jars.ivySettings")
+    ConfigBuilder(MavenUtils.JAR_IVY_SETTING_PATH_KEY)

Review Comment:
   Isn't it more appropriate to have `MavenUtils` refer to this config entry and get the key name?
   
   It seems weird and breaks with the remaining configs for this config to get its key name from somewhere else.



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/util/IvyTestUtils.scala:
##########
@@ -230,12 +226,11 @@ private[deploy] object IvyTestUtils {
       val inside = deps.map(ivyArtifactWriter).mkString("\n")
       "\n  <dependencies>\n" + inside + "\n  </dependencies>"
     }.getOrElse("")
-    content += "\n</ivy-module>"

Review Comment:
   Why do we need to delete this line? It will make the content of `ivy.xml` incomplete...
   
   



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/util/IvyTestUtils.scala:
##########
@@ -230,12 +226,11 @@ private[deploy] object IvyTestUtils {
       val inside = deps.map(ivyArtifactWriter).mkString("\n")
       "\n  <dependencies>\n" + inside + "\n  </dependencies>"
     }.getOrElse("")
-    content += "\n</ivy-module>"

Review Comment:
   Give a fix: https://github.com/apache/spark/pull/43834



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/ArtifactSuite.scala:
##########
@@ -268,4 +269,14 @@ class ArtifactSuite extends ConnectFunSuite with BeforeAndAfterEach {
     val receivedRequests = service.getAndClearLatestAddArtifactRequests()
     assert(receivedRequests.size == 1)
   }
+
+  test("resolve ivy") {
+    val artifacts =
+      Artifact.newIvyArtifacts(URI.create("ivy://org.apache.hive:hive-storage-api:2.7.0"))

Review Comment:
   Is there a risk that we hammer an external repo 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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/util/IvyTestUtils.scala:
##########
@@ -330,58 +341,78 @@ private[deploy] object IvyTestUtils {
 
   /**
    * Creates a suite of jars and poms, with or without dependencies, mocking a maven repository.
-   * @param artifact The main maven coordinate to generate the jar and pom for.
-   * @param dependencies List of dependencies this artifact might have to also create jars and poms.
-   * @param rootDir The root folder of the repository (like `~/.m2/repositories`)
-   * @param useIvyLayout whether to mock the Ivy layout for local repository testing
-   * @param withPython Whether to pack python files inside the jar for extensive testing.
-   * @return Root path of the repository. Will be `rootDir` if supplied.
+   * @param artifact
+   *   The main maven coordinate to generate the jar and pom for.
+   * @param dependencies
+   *   List of dependencies this artifact might have to also create jars and poms.
+   * @param rootDir
+   *   The root folder of the repository (like `~/.m2/repositories`)
+   * @param useIvyLayout
+   *   whether to mock the Ivy layout for local repository testing
+   * @param withPython
+   *   Whether to pack python files inside the jar for extensive testing.
+   * @return
+   *   Root path of the repository. Will be `rootDir` if supplied.
    */
-  private[deploy] def createLocalRepositoryForTests(
+  private[spark] def createLocalRepositoryForTests(
       artifact: MavenCoordinate,
       dependencies: Option[String],
       rootDir: Option[File],
       useIvyLayout: Boolean = false,
       withPython: Boolean = false,
       withR: Boolean = false): File = {
-    val deps = dependencies.map(SparkSubmitUtils.extractMavenCoordinates)
+    val deps = dependencies.map(MavenUtils.extractMavenCoordinates)
     val mainRepo = createLocalRepository(artifact, deps, rootDir, useIvyLayout, withPython, withR)
-    deps.foreach { seq => seq.foreach { dep =>
-      createLocalRepository(dep, None, Some(mainRepo), useIvyLayout, withPython = false)
-    }}
+    deps.foreach { seq =>
+      seq.foreach { dep =>
+        createLocalRepository(dep, None, Some(mainRepo), useIvyLayout, withPython = false)
+      }
+    }
     mainRepo
   }
 
   /**
    * Creates a repository for a test, and cleans it up afterwards.
    *
-   * @param artifact The main maven coordinate to generate the jar and pom for.
-   * @param dependencies List of dependencies this artifact might have to also create jars and poms.
-   * @param rootDir The root folder of the repository (like `~/.m2/repositories`)
-   * @param useIvyLayout whether to mock the Ivy layout for local repository testing
-   * @param withPython Whether to pack python files inside the jar for extensive testing.
-   * @return Root path of the repository. Will be `rootDir` if supplied.
+   * @param artifact
+   *   The main maven coordinate to generate the jar and pom for.
+   * @param dependencies
+   *   List of dependencies this artifact might have to also create jars and poms.
+   * @param rootDir
+   *   The root folder of the repository (like `~/.m2/repositories`)
+   * @param useIvyLayout
+   *   whether to mock the Ivy layout for local repository testing
+   * @param withPython
+   *   Whether to pack python files inside the jar for extensive testing.
+   * @return
+   *   Root path of the repository. Will be `rootDir` if supplied.
    */
-  private[deploy] def withRepository(
+  private[spark] def withRepository(
       artifact: MavenCoordinate,
       dependencies: Option[String],
       rootDir: Option[File],
       useIvyLayout: Boolean = false,
       withPython: Boolean = false,
       withR: Boolean = false,
       ivySettings: IvySettings = new IvySettings)(f: String => Unit): Unit = {
-    val deps = dependencies.map(SparkSubmitUtils.extractMavenCoordinates)
+    val deps = dependencies.map(MavenUtils.extractMavenCoordinates)
     purgeLocalIvyCache(artifact, deps, ivySettings)
-    val repo = createLocalRepositoryForTests(artifact, dependencies, rootDir, useIvyLayout,

Review Comment:
   Did you run scalafmt against this whole file? Let's avoid doing that. This Ivy has a bunch of legacy code, and should be best to avoid unrelated changes 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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/test/scala/org/apache/spark/util/MavenUtilsSuite.scala:
##########
@@ -248,15 +252,18 @@ class SparkSubmitUtilsSuite extends SparkFunSuite {
 
     val settingsFile = Paths.get(tempIvyPath, "ivysettings.xml")
     Files.write(settingsFile, settingsText.getBytes(StandardCharsets.UTF_8))
-    val settings = SparkSubmitUtils.loadIvySettings(settingsFile.toString, None, None)
-    settings.setDefaultIvyUserDir(new File(tempIvyPath))  // NOTE - can't set this through file
+    val settings = MavenUtils.loadIvySettings(settingsFile.toString, None, None)
+    settings.setDefaultIvyUserDir(new File(tempIvyPath)) // NOTE - can't set this through file
 
     val testUtilSettings = new IvySettings
     testUtilSettings.setDefaultIvyUserDir(new File(tempIvyPath))
     IvyTestUtils.withRepository(main, Some(dep), Some(dummyIvyLocal), useIvyLayout = true,
       ivySettings = testUtilSettings) { repo =>
-      val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, settings,
-        transitive = true, isTest = true)
+      val jarPath = MavenUtils.resolveMavenCoordinates(
+        main.toString,
+        settings,
+        transitive = true,
+        isTest = true)

Review Comment:
   I know it's troublesome but let's just revert all these formatting changes.



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

To unsubscribe, e-mail: 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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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

   @vsevolodstep-db 
   
   I found that after moving MavenUtilsSuite.scala to the common-utils module, it cannot pass the test. Do you know why? The current GA does not test this case (this issue will be fixed later), and local reproduction can be executed by `build/sbt "common-utils/test"`.
   
   then
   
   ```
   [info] MavenUtilsSuite:
   [info] - incorrect maven coordinate throws error (8 milliseconds)
   [info] - create repo resolvers (24 milliseconds)
   [info] - create additional resolvers (3 milliseconds)
   :: loading settings :: url = jar:file:/Users/yangjie01/Library/Caches/Coursier/v1/https/repo1.maven.org/maven2/org/apache/ivy/ivy/2.5.1/ivy-2.5.1.jar!/org/apache/ivy/core/settings/ivysettings.xml
   [info] - add dependencies works correctly (35 milliseconds)
   [info] - excludes works correctly (2 milliseconds)
   [info] - ivy path works correctly (3 seconds, 759 milliseconds)
   [info] - search for artifact at local repositories *** FAILED *** (2 seconds, 833 milliseconds)
   [info]   java.lang.RuntimeException: [unresolved dependency: my.great.lib#mylib;0.1: java.text.ParseException: [[Fatal Error] ivy-0.1.xml.original:22:18: XML 文档结构必须从头至尾包含在同一个实体内。 in f/SourceCode/git/spark-mine-sbt/target/tmp/ivy-8b860aca-a9c4-4af9-b15a-ac8c6049b773/cache/my.great.lib/mylib/ivy-0.1.xml.original
   [info] ]]
   [info]   at org.apache.spark.util.MavenUtils$.resolveMavenCoordinates(MavenUtils.scala:459)
   [info]   at org.apache.spark.util.MavenUtilsSuite.$anonfun$new$25(MavenUtilsSuite.scala:173)
   [info]   at org.apache.spark.util.MavenUtilsSuite.$anonfun$new$25$adapted(MavenUtilsSuite.scala:172)
   [info]   at org.apache.spark.util.IvyTestUtils$.withRepository(IvyTestUtils.scala:373)
   [info]   at org.apache.spark.util.MavenUtilsSuite.$anonfun$new$18(MavenUtilsSuite.scala:172)
   [info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
   [info]   at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
   [info]   at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
   [info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
   [info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
   [info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
   [info]   at org.scalatest.TestSuite.withFixture(TestSuite.scala:196)
   [info]   at org.scalatest.TestSuite.withFixture$(TestSuite.scala:195)
   [info]   at org.scalatest.funsuite.AnyFunSuite.withFixture(AnyFunSuite.scala:1564)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
   [info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
   [info]   at org.scalatest.funsuite.AnyFunSuite.runTest(AnyFunSuite.scala:1564)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
   [info]   at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
   [info]   at scala.collection.immutable.List.foreach(List.scala:333)
   [info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
   [info]   at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
   [info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:269)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:268)
   [info]   at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1564)
   [info]   at org.scalatest.Suite.run(Suite.scala:1114)
   [info]   at org.scalatest.Suite.run$(Suite.scala:1096)
   [info]   at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1564)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:273)
   [info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:273)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:272)
   [info]   at org.apache.spark.util.MavenUtilsSuite.org$scalatest$BeforeAndAfterAll$$super$run(MavenUtilsSuite.scala:36)
   [info]   at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
   [info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
   [info]   at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
   [info]   at org.apache.spark.util.MavenUtilsSuite.run(MavenUtilsSuite.scala:36)
   [info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:321)
   [info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:517)
   [info]   at sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:414)
   [info]   at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
   [info]   at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
   [info]   at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
   [info]   at java.base/java.lang.Thread.run(Thread.java:833)
   [info] - dependency not found throws RuntimeException (2 seconds, 274 milliseconds)
   [info] - neglects Spark and Spark's dependencies (1 second, 934 milliseconds)
   [info] - exclude dependencies end to end (953 milliseconds)
   :: loading settings :: file = /Users/yangjie01/SourceCode/git/spark-mine-sbt/target/tmp/ivy-8b860aca-a9c4-4af9-b15a-ac8c6049b773/ivysettings.xml
   [info] - load ivy settings file *** FAILED *** (167 milliseconds)
   [info]   java.lang.RuntimeException: [unresolved dependency: my.great.lib#mylib;0.1: java.text.ParseException: [[Fatal Error] ivy-0.1.xml.original:22:18: XML 文档结构必须从头至尾包含在同一个实体内。 in f/SourceCode/git/spark-mine-sbt/target/tmp/ivy-8b860aca-a9c4-4af9-b15a-ac8c6049b773/cache/my.great.lib/mylib/ivy-0.1.xml.original
   [info] ]]
   [info]   at org.apache.spark.util.MavenUtils$.resolveMavenCoordinates(MavenUtils.scala:459)
   [info]   at org.apache.spark.util.MavenUtilsSuite.$anonfun$new$40(MavenUtilsSuite.scala:260)
   [info]   at org.apache.spark.util.MavenUtilsSuite.$anonfun$new$40$adapted(MavenUtilsSuite.scala:259)
   [info]   at org.apache.spark.util.IvyTestUtils$.withRepository(IvyTestUtils.scala:373)
   [info]   at org.apache.spark.util.MavenUtilsSuite.$anonfun$new$39(MavenUtilsSuite.scala:259)
   [info]   at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)
   [info]   at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
   [info]   at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
   [info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
   [info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
   [info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226)
   [info]   at org.scalatest.TestSuite.withFixture(TestSuite.scala:196)
   [info]   at org.scalatest.TestSuite.withFixture$(TestSuite.scala:195)
   [info]   at org.scalatest.funsuite.AnyFunSuite.withFixture(AnyFunSuite.scala:1564)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236)
   [info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218)
   [info]   at org.scalatest.funsuite.AnyFunSuite.runTest(AnyFunSuite.scala:1564)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269)
   [info]   at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413)
   [info]   at scala.collection.immutable.List.foreach(List.scala:333)
   [info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
   [info]   at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396)
   [info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:269)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:268)
   [info]   at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1564)
   [info]   at org.scalatest.Suite.run(Suite.scala:1114)
   [info]   at org.scalatest.Suite.run$(Suite.scala:1096)
   [info]   at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1564)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:273)
   [info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:535)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:273)
   [info]   at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:272)
   [info]   at org.apache.spark.util.MavenUtilsSuite.org$scalatest$BeforeAndAfterAll$$super$run(MavenUtilsSuite.scala:36)
   [info]   at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
   [info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
   [info]   at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
   [info]   at org.apache.spark.util.MavenUtilsSuite.run(MavenUtilsSuite.scala:36)
   [info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:321)
   [info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:517)
   [info]   at sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:414)
   [info]   at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
   [info]   at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
   [info]   at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
   [info]   at java.base/java.lang.Thread.run(Thread.java:833)
   [info] - SPARK-10878: test resolution files cleaned after resolving artifact (880 milliseconds)
   [info] - SPARK-34624: should ignore non-jar dependencies (145 milliseconds)
   [info] Run completed in 13 seconds, 493 milliseconds.
   [info] Total number of tests run: 13
   [info] Suites: completed 1, aborted 0
   [info] Tests: succeeded 11, failed 2, canceled 0, ignored 0, pending 0
   [info] *** 2 TESTS FAILED ***
   [error] Failed tests:
   [error]         org.apache.spark.util.MavenUtilsSuite
   
   ```


-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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

   Yeah, the test failure at https://github.com/vsevolodstep-db/spark/actions/runs/6691537121/job/18179083580 seems unrelated.


-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell closed pull request #43354: [SPARK-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact
URL: https://github.com/apache/spark/pull/43354


-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/util/SparkTestUtils.scala:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.File
+import java.net.{URI, URL}
+import java.nio.file.Files
+import java.util.Arrays
+import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider}
+
+import scala.jdk.CollectionConverters._
+
+trait SparkTestUtils {
+  // Adapted from the JavaCompiler.java doc examples
+  private val SOURCE = JavaFileObject.Kind.SOURCE
+
+  private def createURI(name: String) = {
+    URI.create(s"string:///${name.replace(".", "/")}${SOURCE.extension}")
+  }
+
+  private[spark] class JavaSourceFromString(val name: String, val code: String)
+      extends SimpleJavaFileObject(createURI(name), SOURCE) {
+    override def getCharContent(ignoreEncodingErrors: Boolean): String = code
+  }
+
+  /** Creates a compiled class with the source file. Class file will be placed in destDir. */
+  def createCompiledClass(

Review Comment:
   Where is this used?



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/util/SparkTestUtils.scala:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.File
+import java.net.{URI, URL}
+import java.nio.file.Files
+import java.util.Arrays
+import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider}
+
+import scala.jdk.CollectionConverters._
+
+trait SparkTestUtils {
+  // Adapted from the JavaCompiler.java doc examples
+  private val SOURCE = JavaFileObject.Kind.SOURCE
+
+  private def createURI(name: String) = {
+    URI.create(s"string:///${name.replace(".", "/")}${SOURCE.extension}")
+  }
+
+  private[spark] class JavaSourceFromString(val name: String, val code: String)
+      extends SimpleJavaFileObject(createURI(name), SOURCE) {
+    override def getCharContent(ignoreEncodingErrors: Boolean): String = code
+  }
+
+  /** Creates a compiled class with the source file. Class file will be placed in destDir. */
+  def createCompiledClass(

Review Comment:
   `IvyTestUtils.createLocalRepository` -> `IvyTestUtils.createJavaClass` to populate a local repository with some JARs.
   Also in `SparkSubmitSuite` and a bunch of others in spark-core



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #43354:
URL: https://github.com/apache/spark/pull/43354#discussion_r1427451350


##########
common/utils/src/main/scala/org/apache/spark/util/IvyTestUtils.scala:
##########
@@ -15,30 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.spark.deploy
+package org.apache.spark.util
 
 import java.io.{File, FileInputStream, FileOutputStream}
-import java.util.jar.{JarEntry, JarOutputStream}
+import java.util.jar.{JarEntry, JarOutputStream, Manifest}
 import java.util.jar.Attributes.Name
-import java.util.jar.Manifest
 
 import scala.collection.mutable.ArrayBuffer
 
-import com.google.common.io.{ByteStreams, Files}
 import org.apache.commons.io.FileUtils
 import org.apache.ivy.core.settings.IvySettings
 
-import org.apache.spark.TestUtils.{createCompiledClass, JavaSourceFromString}
-import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate
-import org.apache.spark.util.Utils
+import org.apache.spark.util.MavenUtils.MavenCoordinate
 
-private[deploy] object IvyTestUtils {
+private[spark] object IvyTestUtils {

Review Comment:
   this file should be in `src/test` instead of `src/main`



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/pom.xml:
##########
@@ -51,6 +51,20 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-text</artifactId>
     </dependency>
+    <dependency>

Review Comment:
   How much deps do we add to the client by doing 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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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


##########
common/utils/pom.xml:
##########
@@ -51,6 +51,20 @@
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-text</artifactId>
     </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.ivy</groupId>
+      <artifactId>ivy</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>oro</groupId>

Review Comment:
   glob matchers (`ivySettings.getMatcher("glob")`) require oro https://ant.apache.org/ivy/history/latest-milestone/concept.html
   
   Also if I remove this dep then tests start failing:
   ```
   [error] Uncaught exception when running org.apache.spark.util.MavenUtilsSuite: java.lang.NoClassDefFoundError: org/apache/oro/text/regex/MalformedPatternException
   [error] sbt.ForkMain$ForkError: java.lang.NoClassDefFoundError: org/apache/oro/text/regex/MalformedPatternException
   [error] 	at org.apache.ivy.plugins.matcher.GlobPatternMatcher.newMatcher(GlobPatternMatcher.java:57)
   [error] 	at org.apache.ivy.plugins.matcher.AbstractPatternMatcher.getMatcher(AbstractPatternMatcher.java:44)
   [error] 	at org.apache.ivy.plugins.matcher.MatcherHelper.matches(MatcherHelper.java:34)
   [error] 	at org.apache.ivy.plugins.matcher.MatcherHelper.matches(MatcherHelper.java:39)
   [error] 	at org.apache.ivy.plugins.matcher.MatcherHelper.matches(MatcherHelper.java:49)
   [error] 	at org.apache.ivy.core.module.descriptor.DefaultModuleDescriptor.doesExclude(DefaultModuleDescriptor.java:783)
   [error] 	at org.apache.ivy.core.resolve.IvyNode.directlyExcludes(IvyNode.java:439)
   [error] 	at org.apache.ivy.core.resolve.IvyNode.doesExclude(IvyNode.java:421)
   [error] 	at org.apache.ivy.core.resolve.IvyNode.isDependencyModuleExcluded(IvyNode.java:411)
   [error] 	at org.apache.ivy.core.resolve.IvyNode.getDependencies(IvyNode.java:361)
   [error] 	at org.apache.ivy.core.resolve.VisitNode.getDependencies(VisitNode.java:305)
   [error] 	at org.apache.ivy.core.resolve.ResolveEngine.doFetchDependencies(ResolveEngine.java:797)
   [error] 	at org.apache.ivy.core.resolve.ResolveEngine.fetchDependencies(ResolveEngine.java:729)
   [error] 	at org.apache.ivy.core.resolve.ResolveEngine.getDependencies(ResolveEngine.java:607)
   [error] 	at org.apache.ivy.core.resolve.ResolveEngine.resolve(ResolveEngine.java:250)
   [error] 	at org.apache.ivy.Ivy.resolve(Ivy.java:522)
   [error] 	at org.apache.spark.util.MavenUtils$.resolveMavenCoordinates(MavenUtils.scala:456)
   ``` 



-- 
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-45506][CONNECT] Add ivy URI support to SparkConnect addArtifact [spark]

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

   @vsevolodstep-db can you retrigger tests? I am not able to do so.


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