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

[GitHub] [spark] vicennial opened a new pull request, #40368: [SPARK-42748] Server-side Artifact Management

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

   <!--
   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
        'core/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.
   -->
   
   This PR adds server-side artifact management as a follow up to the client-side artifact transfer introduced in https://github.com/apache/spark/pull/40256.
   
   Note: The artifacts added on the server are visible to **all users** of the cluster. This is a limitation of the current spark architecture.
   
   ### 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.
   -->
   
   https://github.com/apache/spark/pull/40256 implements the client-side transfer of artifacts to the server but currently, the server does not process these requests.
   
   We need to implement a server-side management mechanism to handle the storage of these artifacts on the driver as well as perform further processing (such as adding jars and moving class files to the right directories).
   
   ### 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'.
   -->
   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.
   -->
   New unit tests.


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

To unsubscribe, e-mail: 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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      this.holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeArtifactToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && !chunkedArtifact.isFinished)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      // Each artifact in the batch is single-chunked.
+      req.getBatch.getArtifactsList.forEach(artifact => writeArtifactToFile(artifact).close())
+    } else {
+      throw new UnsupportedOperationException(s"Unsupported data transfer request: $req")
+    }
+  }
+
+  override def onError(throwable: Throwable): Unit = {
+    Utils.deleteRecursively(stagingDir.toFile)
+    responseObserver.onError(throwable)
+  }
+
+  protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+    artifactManager.addArtifact(holder.session, artifact.path, artifact.stagedPath)
+  }
+
+  /**
+   * Process all the staged artifacts built in this stream.
+   *
+   * @return
+   */
+  protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+    // Non-lazy transformation when using Buffer.
+    stagedArtifacts.map { artifact =>
+      // We do not store artifacts that fail the CRC. The failure is reported in the artifact
+      // summary and it is up to the client to decide whether to retry sending the artifact.
+      if (artifact.getCrcStatus.contains(true)) {
+        addStagedArtifactToArtifactManager(artifact)
+      }
+      artifact.summary()
+    }.toSeq
+  }
+
+  protected def cleanUpStagedArtifacts(): Unit = Utils.deleteRecursively(stagingDir.toFile)
+
+  override def onCompleted(): Unit = {
+    val artifactSummaries = flushStagedArtifacts()
+    // Add the artifacts to the session.

Review Comment:
   Return the summaries to the 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


[GitHub] [spark] LuciferYang commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/ArtifactManagerSuite.scala:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.connect.artifact
+
+import java.nio.file.Paths
+
+import org.apache.commons.io.FileUtils
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connect.ResourceHelper
+import org.apache.spark.sql.connect.service.SparkConnectService
+import org.apache.spark.sql.functions.col
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.util.Utils
+
+class ArtifactManagerSuite extends SharedSparkSession with ResourceHelper {

Review Comment:
   Already create SPARK-43292



-- 
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 pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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

   @vicennial can you update?


-- 
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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -386,6 +387,18 @@ class SparkContext(config: SparkConf) extends Logging {
     Utils.setLogLevel(Level.toLevel(upperCased))
   }
 
+  // Local storage for transferred artifacts through Spark Connect.
+  private val _sparkConnectArtifactDirectory: File = Utils.createTempDir("artifacts")
+
+  /**
+   * :: Experimental ::
+   * Returns the directory that stores artifacts transferred through Spark Connect.
+   *
+   * @since 3.4.0
+   */
+  @Experimental

Review Comment:
   This is primarily for access from the `connect` package but also carries a side benefit of allowing users to inspect the artifact directory. This is a temporary step for now (to pass information to the upper SQL layer) but it may be removed in the longer term (if we switch to passing information down to core from the SQL layer).
   
   No requirement for making it user-facing at the moment so any suggestions to limit scope (while still being able to access it from the `connect` package) would be welcome.



-- 
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] LuciferYang commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/ArtifactManagerSuite.scala:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.connect.artifact
+
+import java.nio.file.Paths
+
+import org.apache.commons.io.FileUtils
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connect.ResourceHelper
+import org.apache.spark.sql.connect.service.SparkConnectService
+import org.apache.spark.sql.functions.col
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.util.Utils
+
+class ArtifactManagerSuite extends SharedSparkSession with ResourceHelper {

Review Comment:
   @vicennial `ArtifactManagerSuite` can't run using maven:
   ```
   build/mvn  clean install -DskipTests -Phive 
   mvn test -pl connector/connect/server  -Phive 
   ```
   
   ```
   23/04/26 16:00:07.666 ScalaTest-main-running-DiscoverySuite ERROR Executor: Could not find org.apache.spark.repl.ExecutorClassLoader on classpath! 
   ```



-- 
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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      val holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+      this.holder = holder
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeDepToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && chunkedArtifact.getRemainingChunks > 0)

Review Comment:
   Removed it in favour of just using `isFinished`



-- 
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] HyukjinKwon commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -386,6 +387,18 @@ class SparkContext(config: SparkConf) extends Logging {
     Utils.setLogLevel(Level.toLevel(upperCased))
   }
 
+  // Local storage for transferred artifacts through Spark Connect.
+  private val _sparkConnectArtifactDirectory: File = Utils.createTempDir("artifacts")
+
+  /**
+   * :: Experimental ::
+   * Returns the directory that stores artifacts transferred through Spark Connect.
+   *
+   * @since 3.4.0
+   */
+  @Experimental
+  def sparkConnectArtifactDirectory: File = _sparkConnectArtifactDirectory

Review Comment:
   maybe ..
   
   ```suggestion
     lazy val sparkConnectArtifactDirectory: File = Utils.createTempDir("artifacts")
   ```



-- 
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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      this.holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeArtifactToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && !chunkedArtifact.isFinished)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      // Each artifact in the batch is single-chunked.
+      req.getBatch.getArtifactsList.forEach(artifact => writeArtifactToFile(artifact).close())
+    } else {
+      throw new UnsupportedOperationException(s"Unsupported data transfer request: $req")
+    }
+  }
+
+  override def onError(throwable: Throwable): Unit = {
+    Utils.deleteRecursively(stagingDir.toFile)
+    responseObserver.onError(throwable)
+  }
+
+  protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+    artifactManager.addArtifact(holder.session, artifact.path, artifact.stagedPath)
+  }
+
+  /**
+   * Process all the staged artifacts built in this stream.
+   *
+   * @return
+   */
+  protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+    // Non-lazy transformation when using Buffer.
+    stagedArtifacts.map { artifact =>
+      // We do not store artifacts that fail the CRC. The failure is reported in the artifact
+      // summary and it is up to the client to decide whether to retry sending the artifact.
+      if (artifact.getCrcStatus.contains(true)) {
+        addStagedArtifactToArtifactManager(artifact)
+      }
+      artifact.summary()
+    }.toSeq
+  }
+
+  protected def cleanUpStagedArtifacts(): Unit = Utils.deleteRecursively(stagingDir.toFile)
+
+  override def onCompleted(): Unit = {
+    val artifactSummaries = flushStagedArtifacts()
+    // Add the artifacts to the session.
+    val builder = proto.AddArtifactsResponse.newBuilder()
+    artifactSummaries.foreach(summary => builder.addArtifacts(summary))
+    // Delete temp dir
+    cleanUpStagedArtifacts()
+
+    // Send the summaries and close
+    responseObserver.onNext(builder.build())
+    responseObserver.onCompleted()
+  }
+
+  /**
+   * Create a (temporary) file for a single-chunk artifact.
+   */
+  private def writeArtifactToFile(
+      artifact: proto.AddArtifactsRequest.SingleChunkArtifact): StagedArtifact = {
+    val stagedDep = new StagedArtifact(artifact.getName)
+    stagedArtifacts += stagedDep
+    stagedDep.write(artifact.getData)
+    stagedDep

Review Comment:
   The `close` here closes both the output stream and builds the artifact summary (essentially, its a "close" for the artifact rather than only the output stream) so there is a bit of logic separation between the "writing" side and the "closing" side (and hence, the write method doesn't take on the additional task).
   It's also consistent for single-chunk and multi-chunk artifacts.



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -386,6 +387,13 @@ class SparkContext(config: SparkConf) extends Logging {
     Utils.setLogLevel(Level.toLevel(upperCased))
   }
 
+  /**
+   * :: Private ::
+   * Returns the directory that stores artifacts transferred through Spark Connect.
+   */
+  @Private
+  lazy val sparkConnectArtifactDirectory: File = Utils.createTempDir("artifacts")

Review Comment:
   Can we scope this down `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


[GitHub] [spark] vicennial commented on pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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

   @hvanhovell updated


-- 
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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      this.holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeArtifactToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && !chunkedArtifact.isFinished)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      // Each artifact in the batch is single-chunked.
+      req.getBatch.getArtifactsList.forEach(artifact => writeArtifactToFile(artifact).close())
+    } else {
+      throw new UnsupportedOperationException(s"Unsupported data transfer request: $req")
+    }
+  }
+
+  override def onError(throwable: Throwable): Unit = {
+    Utils.deleteRecursively(stagingDir.toFile)
+    responseObserver.onError(throwable)
+  }
+
+  protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+    artifactManager.addArtifact(holder.session, artifact.path, artifact.stagedPath)
+  }
+
+  /**
+   * Process all the staged artifacts built in this stream.
+   *
+   * @return
+   */
+  protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+    // Non-lazy transformation when using Buffer.
+    stagedArtifacts.map { artifact =>
+      // We do not store artifacts that fail the CRC. The failure is reported in the artifact
+      // summary and it is up to the client to decide whether to retry sending the artifact.
+      if (artifact.getCrcStatus.contains(true)) {
+        addStagedArtifactToArtifactManager(artifact)
+      }
+      artifact.summary()
+    }.toSeq
+  }
+
+  protected def cleanUpStagedArtifacts(): Unit = Utils.deleteRecursively(stagingDir.toFile)
+
+  override def onCompleted(): Unit = {
+    val artifactSummaries = flushStagedArtifacts()
+    // Add the artifacts to the session.

Review Comment:
   They're being returned [below](https://github.com/apache/spark/pull/40368/files/0d31f1173d9b0597d02bd2b1cd929efae343f495#diff-b2937f1f57c4d43f3c1d42c9f623ea3ccae459c6c331cee0194fb711c638c896R112-R118)
   
   ```
       artifactSummaries.foreach(summary => builder.addArtifacts(summary))
       // Delete temp dir
       cleanUpStagedArtifacts()
   
       // Send the summaries and close
       responseObserver.onNext(builder.build())
       responseObserver.onCompleted()
   ```



-- 
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] LuciferYang commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/test/scala/org/apache/spark/sql/connect/artifact/ArtifactManagerSuite.scala:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.connect.artifact
+
+import java.nio.file.Paths
+
+import org.apache.commons.io.FileUtils
+
+import org.apache.spark.SparkConf
+import org.apache.spark.sql.connect.ResourceHelper
+import org.apache.spark.sql.connect.service.SparkConnectService
+import org.apache.spark.sql.functions.col
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.util.Utils
+
+class ArtifactManagerSuite extends SharedSparkSession with ResourceHelper {

Review Comment:
   https://github.com/apache/spark/pull/40956 to fix 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


[GitHub] [spark] hvanhovell commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      val holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+      this.holder = holder
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeDepToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && chunkedArtifact.getRemainingChunks > 0)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      req.getBatch.getArtifactsList.forEach { artifact =>
+        // Each artifact in the batch is single-chunked.
+        val out = writeDepToFile(artifact)

Review Comment:
   NIT: `writeDepToFile(artifact).close()`



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      val holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+      this.holder = holder
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeDepToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && chunkedArtifact.getRemainingChunks > 0)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      req.getBatch.getArtifactsList.forEach { artifact =>
+        // Each artifact in the batch is single-chunked.
+        val out = writeDepToFile(artifact)
+        out.close()
+      }
+    } else {
+      throw new UnsupportedOperationException(
+        s"$req could not be processed due to unsupported" +
+          s" data transfer mechanism")
+    }
+  }
+
+  override def onError(throwable: Throwable): Unit = {
+    Utils.deleteRecursively(stagingDir.toFile)
+    responseObserver.onError(throwable)
+  }
+
+  protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+    artifactManager.addArtifact(holder.session, artifact.path, artifact.stagedPath)
+  }
+
+  /**
+   * Process all the staged artifacts built in this stream.
+   *
+   * @return
+   */
+  protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+    stagedArtifacts.map { artifact =>

Review Comment:
   You need to be a bit careful with this kind of code. When operating on a Seq `map` can actually be lazy. In this case you are fine because you are using a buffer.



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      this.holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeArtifactToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && !chunkedArtifact.isFinished)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      // Each artifact in the batch is single-chunked.
+      req.getBatch.getArtifactsList.forEach(artifact => writeArtifactToFile(artifact).close())
+    } else {
+      throw new UnsupportedOperationException(s"Unsupported data transfer request: $req")
+    }
+  }
+
+  override def onError(throwable: Throwable): Unit = {
+    Utils.deleteRecursively(stagingDir.toFile)
+    responseObserver.onError(throwable)
+  }
+
+  protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+    artifactManager.addArtifact(holder.session, artifact.path, artifact.stagedPath)
+  }
+
+  /**
+   * Process all the staged artifacts built in this stream.
+   *
+   * @return
+   */
+  protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+    // Non-lazy transformation when using Buffer.
+    stagedArtifacts.map { artifact =>
+      // We do not store artifacts that fail the CRC. The failure is reported in the artifact
+      // summary and it is up to the client to decide whether to retry sending the artifact.
+      if (artifact.getCrcStatus.contains(true)) {
+        addStagedArtifactToArtifactManager(artifact)
+      }
+      artifact.summary()
+    }.toSeq
+  }
+
+  protected def cleanUpStagedArtifacts(): Unit = Utils.deleteRecursively(stagingDir.toFile)
+
+  override def onCompleted(): Unit = {
+    val artifactSummaries = flushStagedArtifacts()
+    // Add the artifacts to the session.
+    val builder = proto.AddArtifactsResponse.newBuilder()
+    artifactSummaries.foreach(summary => builder.addArtifacts(summary))
+    // Delete temp dir
+    cleanUpStagedArtifacts()
+
+    // Send the summaries and close
+    responseObserver.onNext(builder.build())
+    responseObserver.onCompleted()
+  }
+
+  /**
+   * Create a (temporary) file for a single-chunk artifact.
+   */
+  private def writeArtifactToFile(
+      artifact: proto.AddArtifactsRequest.SingleChunkArtifact): StagedArtifact = {
+    val stagedDep = new StagedArtifact(artifact.getName)
+    stagedArtifacts += stagedDep
+    stagedDep.write(artifact.getData)
+    stagedDep

Review Comment:
   You could close it right away. I am not sure what that brings you; it might be easier to reason about.



-- 
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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      this.holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeArtifactToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && !chunkedArtifact.isFinished)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      // Each artifact in the batch is single-chunked.
+      req.getBatch.getArtifactsList.forEach(artifact => writeArtifactToFile(artifact).close())
+    } else {
+      throw new UnsupportedOperationException(s"Unsupported data transfer request: $req")
+    }
+  }
+
+  override def onError(throwable: Throwable): Unit = {
+    Utils.deleteRecursively(stagingDir.toFile)
+    responseObserver.onError(throwable)
+  }
+
+  protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+    artifactManager.addArtifact(holder.session, artifact.path, artifact.stagedPath)
+  }
+
+  /**
+   * Process all the staged artifacts built in this stream.
+   *
+   * @return
+   */
+  protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+    // Non-lazy transformation when using Buffer.
+    stagedArtifacts.map { artifact =>
+      // We do not store artifacts that fail the CRC. The failure is reported in the artifact
+      // summary and it is up to the client to decide whether to retry sending the artifact.
+      if (artifact.getCrcStatus.contains(true)) {
+        addStagedArtifactToArtifactManager(artifact)
+      }
+      artifact.summary()
+    }.toSeq
+  }
+
+  protected def cleanUpStagedArtifacts(): Unit = Utils.deleteRecursively(stagingDir.toFile)
+
+  override def onCompleted(): Unit = {
+    val artifactSummaries = flushStagedArtifacts()
+    // Add the artifacts to the session.

Review Comment:
   Ahh, okay 👍 



-- 
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 pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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

   Merging this one.


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

To unsubscribe, e-mail: 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] vicennial commented on pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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

   PR is mergeable @hvanhovell 


-- 
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] HyukjinKwon commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -386,6 +387,18 @@ class SparkContext(config: SparkConf) extends Logging {
     Utils.setLogLevel(Level.toLevel(upperCased))
   }
 
+  // Local storage for transferred artifacts through Spark Connect.
+  private val _sparkConnectArtifactDirectory: File = Utils.createTempDir("artifacts")
+
+  /**
+   * :: Experimental ::
+   * Returns the directory that stores artifacts transferred through Spark Connect.
+   *
+   * @since 3.4.0
+   */
+  @Experimental

Review Comment:
   Is this a user-facing API?



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

To unsubscribe, e-mail: 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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      val holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+      this.holder = holder
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeDepToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && chunkedArtifact.getRemainingChunks > 0)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      req.getBatch.getArtifactsList.forEach { artifact =>
+        // Each artifact in the batch is single-chunked.
+        val out = writeDepToFile(artifact)
+        out.close()
+      }
+    } else {
+      throw new UnsupportedOperationException(
+        s"$req could not be processed due to unsupported" +

Review Comment:
   s`Unsupported data transfer request: $req`



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      val holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+      this.holder = holder
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeDepToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && chunkedArtifact.getRemainingChunks > 0)

Review Comment:
   OCD comment: `getRemainingChunks` could also be `hasChunksRemaining`.



-- 
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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -465,9 +473,24 @@ class SparkContext(config: SparkConf) extends Logging {
     SparkEnv.set(_env)
 
     // If running the REPL, register the repl's output dir with the file server.
-    _conf.getOption("spark.repl.class.outputDir").foreach { path =>
-      val replUri = _env.rpcEnv.fileServer.addDirectory("/classes", new File(path))
-      _conf.set("spark.repl.class.uri", replUri)
+    _conf.getOption("spark.repl.class.outputDir") match {

Review Comment:
   `DRY`-ed



-- 
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] HyukjinKwon commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -386,6 +387,18 @@ class SparkContext(config: SparkConf) extends Logging {
     Utils.setLogLevel(Level.toLevel(upperCased))
   }
 
+  // Local storage for transferred artifacts through Spark Connect.
+  private val _sparkConnectArtifactDirectory: File = Utils.createTempDir("artifacts")
+
+  /**
+   * :: Experimental ::
+   * Returns the directory that stores artifacts transferred through Spark Connect.
+   *
+   * @since 3.4.0
+   */
+  @Experimental

Review Comment:
   I don't think we should make it user-facing. If the accessor has to be public, we could mark it with `@Private`.



-- 
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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath

Review Comment:
   Makes it easier for testing to inspect the state (used in `TestAddArtifactsHandler` [here](https://github.com/apache/spark/blob/4bb72960465e863986c10ba381e8d9ac27009151/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala#L46))



-- 
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 closed pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

Posted by "hvanhovell (via GitHub)" <gi...@apache.org>.
hvanhovell closed pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management
URL: https://github.com/apache/spark/pull/40368


-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.connect.artifact
+
+import java.net.{URL, URLClassLoader}
+import java.nio.file.{Files, Path, Paths, StandardCopyOption}
+import java.util.concurrent.CopyOnWriteArrayList
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.util.Utils
+
+/**
+ * The Artifact Manager for the [[SparkConnectService]].
+ *
+ * This class handles the storage of artifacts as well as preparing the artifacts for use.
+ * Currently, jars and classfile artifacts undergo additional processing:
+ *   - Jars are automatically added to the underlying [[SparkContext]] and are accessible by all
+ *     users of the cluster.
+ *   - Class files are moved into a common directory that is shared among all users of the
+ *     cluster. Note: Under a multi-user setup, class file conflicts may occur between user
+ *     classes as the class file directory is shared.
+ */
+class SparkConnectArtifactManager private[connect] {
+
+  // The base directory where all artifacts are stored.
+  // Note: If a REPL is attached to the cluster, class file artifacts are stored in the
+  // REPL's output directory.
+  private[connect] lazy val artifactRootPath = SparkContext.getActive match {
+    case Some(sc) =>
+      sc.sparkConnectArtifactDirectory.toPath
+    case None =>
+      throw new RuntimeException("SparkContext is uninitialized!")
+  }
+  private[connect] lazy val artifactRootURI = {
+    val fileServer = SparkEnv.get.rpcEnv.fileServer
+    fileServer.addDirectory("artifacts", artifactRootPath.toFile)
+  }
+
+  // The base directory where all class files are stored.
+  // Note: If a REPL is attached to the cluster, we piggyback on the existing REPL output
+  // directory to store class file artifacts.
+  private[connect] lazy val classArtifactDir = SparkEnv.get.conf
+    .getOption("spark.repl.class.outputDir")
+    .map(p => Paths.get(p))
+    .getOrElse(artifactRootPath.resolve("classes"))
+
+  private[connect] lazy val classArtifactUri: String =
+    SparkEnv.get.conf.getOption("spark.repl.class.uri") match {
+      case Some(uri) => uri
+      case None =>
+        throw new RuntimeException("Class artifact URI had not been initialised in SparkContext!")
+    }
+
+  private val jarsList = new CopyOnWriteArrayList[Path]
+
+  /**
+   * Get the URLs of all jar artifacts added through the [[SparkConnectService]].
+   *
+   * @return
+   */
+  def getSparkConnectAddedJars: Seq[URL] = jarsList.asScala.map(_.toUri.toURL).toSeq
+
+  /**
+   * Add and prepare a staged artifact (i.e an artifact that has been rebuilt locally from bytes
+   * over the wire) for use.
+   *
+   * @param session
+   * @param remoteRelativePath
+   * @param serverLocalStagingPath
+   */
+  private[connect] def addArtifact(
+      session: SparkSession,
+      remoteRelativePath: Path,
+      serverLocalStagingPath: Path): Unit = {
+    require(!remoteRelativePath.isAbsolute)
+    if (remoteRelativePath.startsWith("classes/")) {
+      // Move class files to common location (shared among all users)
+      val target = classArtifactDir.resolve(remoteRelativePath.toString.stripPrefix("classes/"))
+      Files.createDirectories(target.getParent)
+      // Allow overwriting class files to capture updates to classes.
+      Files.move(serverLocalStagingPath, target, StandardCopyOption.REPLACE_EXISTING)
+    } else {
+      val target = artifactRootPath.resolve(remoteRelativePath)
+      Files.createDirectories(target.getParent)
+      // Disallow overwriting jars because spark doesn't support removing jars that were
+      // previously added,
+      if (!Files.exists(target)) {

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


[GitHub] [spark] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -386,6 +387,18 @@ class SparkContext(config: SparkConf) extends Logging {
     Utils.setLogLevel(Level.toLevel(upperCased))
   }
 
+  // Local storage for transferred artifacts through Spark Connect.
+  private val _sparkConnectArtifactDirectory: File = Utils.createTempDir("artifacts")
+
+  /**
+   * :: Experimental ::
+   * Returns the directory that stores artifacts transferred through Spark Connect.
+   *
+   * @since 3.4.0
+   */
+  @Experimental

Review Comment:
   Sounds good 👍 



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.connect.artifact
+
+import java.net.{URL, URLClassLoader}
+import java.nio.file.{Files, Path, Paths, StandardCopyOption}
+import java.util.concurrent.CopyOnWriteArrayList
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.util.Utils
+
+/**
+ * The Artifact Manager for the [[SparkConnectService]].
+ *
+ * This class handles the storage of artifacts as well as preparing the artifacts for use.
+ * Currently, jars and classfile artifacts undergo additional processing:
+ *   - Jars are automatically added to the underlying [[SparkContext]] and are accessible by all
+ *     users of the cluster.
+ *   - Class files are moved into a common directory that is shared among all users of the
+ *     cluster. Note: Under a multi-user setup, class file conflicts may occur between user
+ *     classes as the class file directory is shared.
+ */
+class SparkConnectArtifactManager private[connect] {
+
+  // The base directory where all artifacts are stored.
+  // Note: If a REPL is attached to the cluster, class file artifacts are stored in the
+  // REPL's output directory.
+  private[connect] lazy val artifactRootPath = SparkContext.getActive match {

Review Comment:
   NIT: just pass in the SparkContext when we start the thing...



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      this.holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeArtifactToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && !chunkedArtifact.isFinished)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      // Each artifact in the batch is single-chunked.
+      req.getBatch.getArtifactsList.forEach(artifact => writeArtifactToFile(artifact).close())
+    } else {
+      throw new UnsupportedOperationException(s"Unsupported data transfer request: $req")
+    }
+  }
+
+  override def onError(throwable: Throwable): Unit = {
+    Utils.deleteRecursively(stagingDir.toFile)
+    responseObserver.onError(throwable)
+  }
+
+  protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+    artifactManager.addArtifact(holder.session, artifact.path, artifact.stagedPath)
+  }
+
+  /**
+   * Process all the staged artifacts built in this stream.
+   *
+   * @return
+   */
+  protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+    // Non-lazy transformation when using Buffer.
+    stagedArtifacts.map { artifact =>
+      // We do not store artifacts that fail the CRC. The failure is reported in the artifact
+      // summary and it is up to the client to decide whether to retry sending the artifact.
+      if (artifact.getCrcStatus.contains(true)) {
+        addStagedArtifactToArtifactManager(artifact)
+      }
+      artifact.summary()
+    }.toSeq
+  }
+
+  protected def cleanUpStagedArtifacts(): Unit = Utils.deleteRecursively(stagingDir.toFile)
+
+  override def onCompleted(): Unit = {
+    val artifactSummaries = flushStagedArtifacts()
+    // Add the artifacts to the session.

Review Comment:
   I mean update the comment :)



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

To unsubscribe, e-mail: 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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/artifact/SparkConnectArtifactManager.scala:
##########
@@ -0,0 +1,159 @@
+/*
+ * 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.connect.artifact
+
+import java.net.{URL, URLClassLoader}
+import java.nio.file.{Files, Path, Paths, StandardCopyOption}
+import java.util.concurrent.CopyOnWriteArrayList
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkContext, SparkEnv}
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.util.Utils
+
+/**
+ * The Artifact Manager for the [[SparkConnectService]].
+ *
+ * This class handles the storage of artifacts as well as preparing the artifacts for use.
+ * Currently, jars and classfile artifacts undergo additional processing:
+ *   - Jars are automatically added to the underlying [[SparkContext]] and are accessible by all
+ *     users of the cluster.
+ *   - Class files are moved into a common directory that is shared among all users of the
+ *     cluster. Note: Under a multi-user setup, class file conflicts may occur between user
+ *     classes as the class file directory is shared.
+ */
+class SparkConnectArtifactManager private[connect] {
+
+  // The base directory where all artifacts are stored.
+  // Note: If a REPL is attached to the cluster, class file artifacts are stored in the
+  // REPL's output directory.
+  private[connect] lazy val artifactRootPath = SparkContext.getActive match {

Review Comment:
   Hmm, `SparkConnectPlanner` callsites can pass this in via the `session` but it gets awkward in `SparkConnectAddArtifactsHandler` where we'll need to create/obtain the active Artifact Manager via a SparkSession lookup from the request



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      val holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+      this.holder = holder

Review Comment:
   Nit directly assign the holder?



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath

Review Comment:
   Why are these protected?



-- 
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 #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
core/src/main/scala/org/apache/spark/SparkContext.scala:
##########
@@ -465,9 +473,24 @@ class SparkContext(config: SparkConf) extends Logging {
     SparkEnv.set(_env)
 
     // If running the REPL, register the repl's output dir with the file server.
-    _conf.getOption("spark.repl.class.outputDir").foreach { path =>
-      val replUri = _env.rpcEnv.fileServer.addDirectory("/classes", new File(path))
-      _conf.set("spark.repl.class.uri", replUri)
+    _conf.getOption("spark.repl.class.outputDir") match {

Review Comment:
   This could be a bit `DRY`er.



-- 
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] vicennial commented on a diff in pull request #40368: [SPARK-42748][CONNECT] Server-side Artifact Management

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


##########
connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala:
##########
@@ -0,0 +1,246 @@
+/*
+ * 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.connect.service
+
+import com.google.common.io.CountingOutputStream
+import io.grpc.stub.StreamObserver
+import java.nio.file.{Files, Path, Paths}
+import java.util.zip.{CheckedOutputStream, CRC32}
+import scala.collection.mutable
+import scala.util.control.NonFatal
+
+import org.apache.spark.connect.proto
+import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse}
+import org.apache.spark.connect.proto.AddArtifactsResponse.ArtifactSummary
+import org.apache.spark.sql.connect.artifact.SparkConnectArtifactManager
+import org.apache.spark.util.Utils
+
+/**
+ * Handles [[AddArtifactsRequest]]s for the [[SparkConnectService]].
+ *
+ * @param responseObserver
+ */
+class SparkConnectAddArtifactsHandler(val responseObserver: StreamObserver[AddArtifactsResponse])
+    extends StreamObserver[AddArtifactsRequest] {
+
+  // Temporary directory where artifacts are rebuilt from the bytes sent over the wire.
+  protected val stagingDir: Path = Utils.createTempDir().toPath
+  protected val stagedArtifacts: mutable.Buffer[StagedArtifact] =
+    mutable.Buffer.empty[StagedArtifact]
+  // If not null, indicates the currently active chunked artifact that is being rebuilt from
+  // several [[AddArtifactsRequest]]s.
+  private var chunkedArtifact: StagedChunkedArtifact = _
+  private var holder: SessionHolder = _
+  private def artifactManager: SparkConnectArtifactManager =
+    SparkConnectArtifactManager.getOrCreateArtifactManager
+
+  override def onNext(req: AddArtifactsRequest): Unit = {
+    if (this.holder == null) {
+      val holder = SparkConnectService.getOrCreateIsolatedSession(
+        req.getUserContext.getUserId,
+        req.getSessionId)
+      this.holder = holder
+    }
+
+    if (req.hasBeginChunk) {
+      // The beginning of a multi-chunk artifact.
+      require(chunkedArtifact == null)
+      chunkedArtifact = writeDepToFile(req.getBeginChunk)
+    } else if (req.hasChunk) {
+      // We are currently processing a multi-chunk artifact
+      require(chunkedArtifact != null && chunkedArtifact.getRemainingChunks > 0)
+      chunkedArtifact.write(req.getChunk)
+
+      if (chunkedArtifact.isFinished) {
+        chunkedArtifact.close()
+        // Unset the currently active chunked artifact.
+        chunkedArtifact = null
+      }
+    } else if (req.hasBatch) {
+      req.getBatch.getArtifactsList.forEach { artifact =>
+        // Each artifact in the batch is single-chunked.
+        val out = writeDepToFile(artifact)
+        out.close()
+      }
+    } else {
+      throw new UnsupportedOperationException(
+        s"$req could not be processed due to unsupported" +
+          s" data transfer mechanism")
+    }
+  }
+
+  override def onError(throwable: Throwable): Unit = {
+    Utils.deleteRecursively(stagingDir.toFile)
+    responseObserver.onError(throwable)
+  }
+
+  protected def addStagedArtifactToArtifactManager(artifact: StagedArtifact): Unit = {
+    artifactManager.addArtifact(holder.session, artifact.path, artifact.stagedPath)
+  }
+
+  /**
+   * Process all the staged artifacts built in this stream.
+   *
+   * @return
+   */
+  protected def flushStagedArtifacts(): Seq[ArtifactSummary] = {
+    stagedArtifacts.map { artifact =>

Review Comment:
   Ack. Left a comment in the method.



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