-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-46812][CONNECT][PYTHON] Make mapInPandas / mapInArrow support ResourceProfile #45232
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 3 commits
11930c9
a7ccd5e
95a75af
ca41951
30531ae
5e61437
ba5c25a
3899ada
0e0f91f
fb6d721
192e59e
1bdb284
7d1523f
7efa33b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -967,6 +967,34 @@ message FetchErrorDetailsResponse { | |
| } | ||
| } | ||
|
|
||
| message BuildResourceProfileRequest { | ||
| // (Required) | ||
| // | ||
| // The session_id specifies a spark session for a user id (which is specified | ||
| // by user_context.user_id). The session_id is set by the client to be able to | ||
| // collate streaming responses from different queries within the dedicated session. | ||
| // The id should be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff` | ||
| string session_id = 1; | ||
|
|
||
| // (Required) User context | ||
| UserContext user_context = 2; | ||
|
|
||
| // (Required) The ResourceProfile to be built on the server-side. | ||
| ResourceProfile profile = 3; | ||
| } | ||
|
|
||
| // Response to building resource profile. | ||
| message BuildResourceProfileResponse { | ||
| string session_id = 1; | ||
|
|
||
| // Server-side generated idempotency key that the client can use to assert that the server side | ||
| // session has not changed. | ||
| string server_side_session_id = 4; | ||
|
|
||
| // (Required) Server-side generated resource profile id. | ||
| int32 profile_id = 5; | ||
| } | ||
|
|
||
| // Main interface for the SparkConnect service. | ||
| service SparkConnectService { | ||
|
|
||
|
|
@@ -1011,5 +1039,7 @@ service SparkConnectService { | |
|
|
||
| // FetchErrorDetails retrieves the matched exception with details based on a provided error id. | ||
| rpc FetchErrorDetails(FetchErrorDetailsRequest) returns (FetchErrorDetailsResponse) {} | ||
| } | ||
|
|
||
| // Build ResourceProfile and get the profile id | ||
| rpc BuildResourceProfile(BuildResourceProfileRequest) returns (BuildResourceProfileResponse) {} | ||
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ import "google/protobuf/any.proto"; | |
| import "spark/connect/expressions.proto"; | ||
| import "spark/connect/types.proto"; | ||
| import "spark/connect/catalog.proto"; | ||
| import "spark/connect/common.proto"; | ||
|
|
||
| option java_multiple_files = true; | ||
| option java_package = "org.apache.spark.connect.proto"; | ||
|
|
@@ -892,6 +893,9 @@ message MapPartitions { | |
|
|
||
| // (Optional) Whether to use barrier mode execution or not. | ||
| optional bool is_barrier = 3; | ||
|
|
||
| // (Optional) ResourceProfile id used for the stage level scheduling. | ||
| optional int32 profile_id = 4; | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shouldn't these be uuids? Just to make sure that we have intentional difference and no off by one?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @grundprinzip, I'm not quite following you about "by uuids". So the basic implementation is
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we need an extra RPC for that? Can't you attach the resource profile directly to this call?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @grundprinzip, The user still needs to know the exact ResourceProfile id, if we attach resource profile in the call, seems we can't get id in this call. |
||
| } | ||
|
|
||
| message GroupMap { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,75 @@ | ||
| /* | ||
| * 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 scala.jdk.CollectionConverters.MapHasAsScala | ||
|
|
||
| import io.grpc.stub.StreamObserver | ||
|
|
||
| import org.apache.spark.connect.proto | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} | ||
|
|
||
| class SparkConnectBuildResourceProfileHandler( | ||
| responseObserver: StreamObserver[proto.BuildResourceProfileResponse]) | ||
| extends Logging { | ||
|
|
||
| /** | ||
| * transform the spark connect ResourceProfile to spark ResourceProfile | ||
| * @param rp | ||
| * Spark connect ResourceProfile | ||
| * @return | ||
| * the Spark ResourceProfile | ||
| */ | ||
| private def transformResourceProfile(rp: proto.ResourceProfile): ResourceProfile = { | ||
| val ereqs = rp.getExecutorResourcesMap.asScala.map { case (name, res) => | ||
| name -> new ExecutorResourceRequest( | ||
| res.getResourceName, | ||
| res.getAmount, | ||
| res.getDiscoveryScript, | ||
| res.getVendor) | ||
| }.toMap | ||
| val treqs = rp.getTaskResourcesMap.asScala.map { case (name, res) => | ||
| name -> new TaskResourceRequest(res.getResourceName, res.getAmount) | ||
| }.toMap | ||
|
|
||
| if (ereqs.isEmpty) { | ||
| new TaskResourceProfile(treqs) | ||
| } else { | ||
| new ResourceProfile(ereqs, treqs) | ||
| } | ||
| } | ||
|
|
||
| def handle(request: proto.BuildResourceProfileRequest): Unit = { | ||
| val holder = SparkConnectService | ||
| .getOrCreateIsolatedSession(request.getUserContext.getUserId, request.getSessionId) | ||
|
|
||
| val rp = transformResourceProfile(request.getProfile) | ||
|
|
||
| val session = holder.session | ||
| session.sparkContext.resourceProfileManager.addResourceProfile(rp) | ||
|
||
|
|
||
| val builder = proto.BuildResourceProfileResponse.newBuilder() | ||
| builder.setProfileId(rp.id) | ||
| builder.setSessionId(request.getSessionId) | ||
| builder.setServerSideSessionId(holder.serverSessionId) | ||
| responseObserver.onNext(builder.build()) | ||
| responseObserver.onCompleted() | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -554,6 +554,7 @@ def __hash__(self): | |
| "pyspark.resource.profile", | ||
| # unittests | ||
| "pyspark.resource.tests.test_resources", | ||
| "pyspark.resource.tests.test_connect_resources", | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this test is for spark connect, I think we should move it to Module or maybe we can move the test cases in it to
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This pull request already includes pyspark.sql.tests.connect.test_resources to test the general mapInPandas/mapInArrow functionality with ResourceProfile. On the other hand, |
||
| ], | ||
| ) | ||
|
|
||
|
|
@@ -1027,6 +1028,7 @@ def __hash__(self): | |
| "pyspark.sql.tests.connect.test_parity_pandas_udf_scalar", | ||
| "pyspark.sql.tests.connect.test_parity_pandas_udf_grouped_agg", | ||
| "pyspark.sql.tests.connect.test_parity_pandas_udf_window", | ||
| "pyspark.sql.tests.connect.test_resources", | ||
| ], | ||
| excluded_python_implementations=[ | ||
| "PyPy" # Skip these tests under PyPy since they require numpy, pandas, and pyarrow and | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This would need @grundprinzip and @hvanhovell 's review.