From eec67b293a4178d071cedc07435ac4bbc4689b34 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 10 Feb 2023 20:16:51 +0800 Subject: [PATCH 01/14] init simplify simplify simplify init init init init --- .../main/protobuf/spark/connect/base.proto | 82 +++++++ .../service/SparkConnectConfigHandler.scala | 142 +++++++++++ .../connect/service/SparkConnectService.scala | 15 ++ python/pyspark/sql/connect/proto/base_pb2.py | 50 +++- python/pyspark/sql/connect/proto/base_pb2.pyi | 222 ++++++++++++++++++ .../sql/connect/proto/base_pb2_grpc.py | 45 ++++ .../apache/spark/sql/internal/SQLConf.scala | 14 +- 7 files changed, 561 insertions(+), 9 deletions(-) create mode 100644 connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 5f9a4411ecdcc..522bc1113a945 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -183,6 +183,85 @@ message ExecutePlanResponse { } } +// Request to update or fetch the configurations. +message ConfigRequest { + // (Required) + // + // The client_id is set by the client to be able to collate streaming responses from + // different queries. + string client_id = 1; + + // (Required) User context + UserContext user_context = 2; + + Operation operation = 3; + + // (Optional) + // + // Identify which keys will be updated or fetched. + // Required when the 'operation' is 'SET', 'GET', 'GET_OPTION', 'UNSET', + // 'CONTAINS', 'IS_MODIFIABLE'. + repeated string keys = 4; + + // (Optional) + // + // Corresponding values to the keys. + // Required when the 'operation' is 'SET'. + // Optional when the 'operation' is 'GET', the values here will be used + // as the default values. + repeated string values = 5; + + // (Optional) + // + // Only used when the 'operation' is 'GET_ALL'. + // If prefix is given, only parameters that start with the prefix will be returned. + optional string prefix = 6; + + enum Operation { + OPERATION_UNSPECIFIED = 0; + OPERATION_SET = 1; + OPERATION_GET = 2; + OPERATION_GET_OPTION = 3; + OPERATION_GET_ALL = 4; + OPERATION_UNSET = 5; + OPERATION_CONTAINS = 6; + OPERATION_IS_MODIFIABLE = 7; + } +} + +message ConfigResponse { + string client_id = 1; + + // (Optional) + // + // Available when the operation is 'GET_ALL'. + repeated string keys = 2; + + // (Optional) + // + // Available when the operation is 'GET', 'GET_ALL'. + repeated string values = 3; + + // (Optional) + // + // Available when the operation is 'GET_OPTION'. + repeated OptionalValue optional_values = 4; + + // (Optional) + // + // Available when the operation is 'CONTAINS', 'IS_MODIFIABLE'. + repeated bool bools = 5; + + // (Optional) + // + // Warning messages for deprecated or unsupported configurations. + repeated string warnings = 6; + + message OptionalValue { + optional string value = 1; + } +} + // Main interface for the SparkConnect service. service SparkConnectService { @@ -193,5 +272,8 @@ service SparkConnectService { // Analyzes a query and returns a [[AnalyzeResponse]] containing metadata about the query. rpc AnalyzePlan(AnalyzePlanRequest) returns (AnalyzePlanResponse) {} + + // Update or fetch the configurations and returns a [[ConfigResponse]] containing the result. + rpc Config(ConfigRequest) returns (ConfigResponse) {} } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala new file mode 100644 index 0000000000000..80f80d76e6539 --- /dev/null +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -0,0 +1,142 @@ +/* + * 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.collection.JavaConverters._ + +import io.grpc.stub.StreamObserver + +import org.apache.spark.connect.proto +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.SQLConf + +class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigResponse]) + extends Logging { + + def handle(request: proto.ConfigRequest): Unit = { + val session = + SparkConnectService + .getOrCreateIsolatedSession(request.getUserContext.getUserId, request.getClientId) + .session + + val builder = handleSQLConf(request, session) + builder.setClientId(request.getClientId) + responseObserver.onNext(builder.build()) + responseObserver.onCompleted() + } + + private def handleSQLConf(request: proto.ConfigRequest, session: SparkSession) = { + val conf = session.conf + val builder = proto.ConfigResponse.newBuilder() + + request.getOperation match { + case proto.ConfigRequest.Operation.OPERATION_SET => + if (request.getKeysCount != request.getValuesCount) { + throw new UnsupportedOperationException("Keys and values should have the same length!") + } + request.getKeysList.asScala.iterator + .zip(request.getValuesList.asScala.iterator) + .foreach { case (key, value) => + conf.set(key, value) + getWarning(key).foreach(builder.addWarnings) + } + + case proto.ConfigRequest.Operation.OPERATION_GET => + if (request.getValuesCount == 0) { + request.getKeysList.asScala.iterator.foreach { key => + builder.addValues(conf.get(key)) + getWarning(key).foreach(builder.addWarnings) + } + } else { + if (request.getKeysCount != request.getValuesCount) { + throw new UnsupportedOperationException( + "Keys and values should have the same length!") + } + request.getKeysList.asScala.iterator + .zip(request.getValuesList.asScala.iterator) + .foreach { case (key, value) => + builder.addValues(conf.get(key, value)) + getWarning(key).foreach(builder.addWarnings) + } + } + + case proto.ConfigRequest.Operation.OPERATION_GET_OPTION => + request.getKeysList.asScala.iterator.foreach { key => + conf.getOption(key) match { + case Some(value) => + builder.addOptionalValues( + proto.ConfigResponse.OptionalValue.newBuilder().setValue(value).build()) + case _ => + builder.addOptionalValues(proto.ConfigResponse.OptionalValue.newBuilder().build()) + } + getWarning(key).foreach(builder.addWarnings) + } + + case proto.ConfigRequest.Operation.OPERATION_GET_ALL => + val results = if (request.hasPrefix) { + val prefix = request.getPrefix + conf.getAll.iterator + .filter { case (key, value) => key.startsWith(prefix) } + .map { case (key, value) => (key.substring(prefix.length), value) } + } else { + conf.getAll.iterator + } + results.foreach { case (key, value) => + builder.addKeys(key).addValues(value) + getWarning(key).foreach(builder.addWarnings) + } + + case proto.ConfigRequest.Operation.OPERATION_UNSET => + request.getKeysList.asScala.iterator.foreach { key => + conf.unset(key) + getWarning(key).foreach(builder.addWarnings) + } + + case proto.ConfigRequest.Operation.OPERATION_CONTAINS => + request.getKeysList.asScala.iterator.foreach { key => + builder.addBools(conf.contains(key)) + getWarning(key).foreach(builder.addWarnings) + } + + case proto.ConfigRequest.Operation.OPERATION_IS_MODIFIABLE => + request.getKeysList.asScala.iterator.foreach { key => + builder.addBools(conf.isModifiable(key)) + getWarning(key).foreach(builder.addWarnings) + } + + case other => + throw new UnsupportedOperationException(s"Unsupported operation $other in SQLConf.") + } + + builder + } + + private def getWarning(key: String): Option[String] = { + if (SparkConnectConfigHandler.unsupportedConfigurations.contains(key)) { + Some(s"The SQL config '$key' is NOT supported in Spark Connect") + } else { + SQLConf.deprecatedSQLConfigs.get(key).map(_.toDeprecationString) + } + } +} + +object SparkConnectConfigHandler { + + private[connect] val unsupportedConfigurations = Set("spark.sql.execution.arrow.enabled") +} diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 959aceaf46a38..227067e2fafb5 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -212,6 +212,21 @@ class SparkConnectService(debug: Boolean) response.setIsStreaming(ds.isStreaming) response.addAllInputFiles(ds.inputFiles.toSeq.asJava) } + + /** + * This is the main entry method for Spark Connect and all calls to update or fetch + * configuration.. + * + * @param request + * @param responseObserver + */ + override def config( + request: proto.ConfigRequest, + responseObserver: StreamObserver[proto.ConfigResponse]): Unit = { + try { + new SparkConnectConfigHandler(responseObserver).handle(request) + } catch handleError("config", observer = responseObserver) + } } /** diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 0d86ce8cd687b..4ee8a5b84243e 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType2\xc7\x01\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"\xcf\x03\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0e\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12\x12\n\x04keys\x18\x04 \x03(\tR\x04keys\x12\x16\n\x06values\x18\x05 \x03(\tR\x06values\x12\x1b\n\x06prefix\x18\x06 \x01(\tH\x00R\x06prefix\x88\x01\x01"\xc7\x01\n\tOperation\x12\x19\n\x15OPERATION_UNSPECIFIED\x10\x00\x12\x11\n\rOPERATION_SET\x10\x01\x12\x11\n\rOPERATION_GET\x10\x02\x12\x18\n\x14OPERATION_GET_OPTION\x10\x03\x12\x15\n\x11OPERATION_GET_ALL\x10\x04\x12\x13\n\x0fOPERATION_UNSET\x10\x05\x12\x16\n\x12OPERATION_CONTAINS\x10\x06\x12\x1b\n\x17OPERATION_IS_MODIFIABLE\x10\x07\x42\t\n\x07_prefix"\x97\x02\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x12\n\x04keys\x18\x02 \x03(\tR\x04keys\x12\x16\n\x06values\x18\x03 \x03(\tR\x06values\x12T\n\x0foptional_values\x18\x04 \x03(\x0b\x32+.spark.connect.ConfigResponse.OptionalValueR\x0eoptionalValues\x12\x14\n\x05\x62ools\x18\x05 \x03(\x08R\x05\x62ools\x12\x1a\n\x08warnings\x18\x06 \x03(\tR\x08warnings\x1a\x34\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -58,7 +58,11 @@ _EXECUTEPLANRESPONSE_METRICS_METRICVALUE = _EXECUTEPLANRESPONSE_METRICS.nested_types_by_name[ "MetricValue" ] +_CONFIGREQUEST = DESCRIPTOR.message_types_by_name["ConfigRequest"] +_CONFIGRESPONSE = DESCRIPTOR.message_types_by_name["ConfigResponse"] +_CONFIGRESPONSE_OPTIONALVALUE = _CONFIGRESPONSE.nested_types_by_name["OptionalValue"] _EXPLAIN_EXPLAINMODE = _EXPLAIN.enum_types_by_name["ExplainMode"] +_CONFIGREQUEST_OPERATION = _CONFIGREQUEST.enum_types_by_name["Operation"] Plan = _reflection.GeneratedProtocolMessageType( "Plan", (_message.Message,), @@ -186,6 +190,38 @@ _sym_db.RegisterMessage(ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntry) _sym_db.RegisterMessage(ExecutePlanResponse.Metrics.MetricValue) +ConfigRequest = _reflection.GeneratedProtocolMessageType( + "ConfigRequest", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest) + }, +) +_sym_db.RegisterMessage(ConfigRequest) + +ConfigResponse = _reflection.GeneratedProtocolMessageType( + "ConfigResponse", + (_message.Message,), + { + "OptionalValue": _reflection.GeneratedProtocolMessageType( + "OptionalValue", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_OPTIONALVALUE, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.OptionalValue) + }, + ), + "DESCRIPTOR": _CONFIGRESPONSE, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse) + }, +) +_sym_db.RegisterMessage(ConfigResponse) +_sym_db.RegisterMessage(ConfigResponse.OptionalValue) + _SPARKCONNECTSERVICE = DESCRIPTOR.services_by_name["SparkConnectService"] if _descriptor._USE_C_DESCRIPTORS == False: @@ -219,6 +255,14 @@ _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 2017 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 2019 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 2107 - _SPARKCONNECTSERVICE._serialized_start = 2110 - _SPARKCONNECTSERVICE._serialized_end = 2309 + _CONFIGREQUEST._serialized_start = 2110 + _CONFIGREQUEST._serialized_end = 2573 + _CONFIGREQUEST_OPERATION._serialized_start = 2363 + _CONFIGREQUEST_OPERATION._serialized_end = 2562 + _CONFIGRESPONSE._serialized_start = 2576 + _CONFIGRESPONSE._serialized_end = 2855 + _CONFIGRESPONSE_OPTIONALVALUE._serialized_start = 2803 + _CONFIGRESPONSE_OPTIONALVALUE._serialized_end = 2855 + _SPARKCONNECTSERVICE._serialized_start = 2858 + _SPARKCONNECTSERVICE._serialized_end = 3130 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index ea82aaf21e252..21feba0e88f48 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -570,3 +570,225 @@ class ExecutePlanResponse(google.protobuf.message.Message): ) -> None: ... global___ExecutePlanResponse = ExecutePlanResponse + +class ConfigRequest(google.protobuf.message.Message): + """Request to update or fetch the configurations.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class _Operation: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + + class _OperationEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + ConfigRequest._Operation.ValueType + ], + builtins.type, + ): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + OPERATION_UNSPECIFIED: ConfigRequest._Operation.ValueType # 0 + OPERATION_SET: ConfigRequest._Operation.ValueType # 1 + OPERATION_GET: ConfigRequest._Operation.ValueType # 2 + OPERATION_GET_OPTION: ConfigRequest._Operation.ValueType # 3 + OPERATION_GET_ALL: ConfigRequest._Operation.ValueType # 4 + OPERATION_UNSET: ConfigRequest._Operation.ValueType # 5 + OPERATION_CONTAINS: ConfigRequest._Operation.ValueType # 6 + OPERATION_IS_MODIFIABLE: ConfigRequest._Operation.ValueType # 7 + + class Operation(_Operation, metaclass=_OperationEnumTypeWrapper): ... + OPERATION_UNSPECIFIED: ConfigRequest.Operation.ValueType # 0 + OPERATION_SET: ConfigRequest.Operation.ValueType # 1 + OPERATION_GET: ConfigRequest.Operation.ValueType # 2 + OPERATION_GET_OPTION: ConfigRequest.Operation.ValueType # 3 + OPERATION_GET_ALL: ConfigRequest.Operation.ValueType # 4 + OPERATION_UNSET: ConfigRequest.Operation.ValueType # 5 + OPERATION_CONTAINS: ConfigRequest.Operation.ValueType # 6 + OPERATION_IS_MODIFIABLE: ConfigRequest.Operation.ValueType # 7 + + CLIENT_ID_FIELD_NUMBER: builtins.int + USER_CONTEXT_FIELD_NUMBER: builtins.int + OPERATION_FIELD_NUMBER: builtins.int + KEYS_FIELD_NUMBER: builtins.int + VALUES_FIELD_NUMBER: builtins.int + PREFIX_FIELD_NUMBER: builtins.int + client_id: builtins.str + """(Required) + + The client_id is set by the client to be able to collate streaming responses from + different queries. + """ + @property + def user_context(self) -> global___UserContext: + """(Required) User context""" + operation: global___ConfigRequest.Operation.ValueType + @property + def keys( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Optional) + + Identify which keys will be updated or fetched. + Required when the 'operation' is 'SET', 'GET', 'GET_OPTION', 'UNSET', + 'CONTAINS', 'IS_MODIFIABLE'. + """ + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Optional) + + Corresponding values to the keys. + Required when the 'operation' is 'SET'. + Optional when the 'operation' is 'GET', the values here will be used + as the default values. + """ + prefix: builtins.str + """(Optional) + + Only used when the 'operation' is 'GET_ALL'. + If prefix is given, only parameters that start with the prefix will be returned. + """ + def __init__( + self, + *, + client_id: builtins.str = ..., + user_context: global___UserContext | None = ..., + operation: global___ConfigRequest.Operation.ValueType = ..., + keys: collections.abc.Iterable[builtins.str] | None = ..., + values: collections.abc.Iterable[builtins.str] | None = ..., + prefix: builtins.str | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_prefix", b"_prefix", "prefix", b"prefix", "user_context", b"user_context" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_prefix", + b"_prefix", + "client_id", + b"client_id", + "keys", + b"keys", + "operation", + b"operation", + "prefix", + b"prefix", + "user_context", + b"user_context", + "values", + b"values", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_prefix", b"_prefix"] + ) -> typing_extensions.Literal["prefix"] | None: ... + +global___ConfigRequest = ConfigRequest + +class ConfigResponse(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class OptionalValue(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUE_FIELD_NUMBER: builtins.int + value: builtins.str + def __init__( + self, + *, + value: builtins.str | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["_value", b"_value", "value", b"value"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["_value", b"_value", "value", b"value"] + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_value", b"_value"] + ) -> typing_extensions.Literal["value"] | None: ... + + CLIENT_ID_FIELD_NUMBER: builtins.int + KEYS_FIELD_NUMBER: builtins.int + VALUES_FIELD_NUMBER: builtins.int + OPTIONAL_VALUES_FIELD_NUMBER: builtins.int + BOOLS_FIELD_NUMBER: builtins.int + WARNINGS_FIELD_NUMBER: builtins.int + client_id: builtins.str + @property + def keys( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Optional) + + Available when the operation is 'GET_ALL'. + """ + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Optional) + + Available when the operation is 'GET', 'GET_ALL'. + """ + @property + def optional_values( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___ConfigResponse.OptionalValue + ]: + """(Optional) + + Available when the operation is 'GET_OPTION'. + """ + @property + def bools( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.bool]: + """(Optional) + + Available when the operation is 'CONTAINS', 'IS_MODIFIABLE'. + """ + @property + def warnings( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Optional) + + Warning messages for deprecated or unsupported configurations. + """ + def __init__( + self, + *, + client_id: builtins.str = ..., + keys: collections.abc.Iterable[builtins.str] | None = ..., + values: collections.abc.Iterable[builtins.str] | None = ..., + optional_values: collections.abc.Iterable[global___ConfigResponse.OptionalValue] + | None = ..., + bools: collections.abc.Iterable[builtins.bool] | None = ..., + warnings: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "bools", + b"bools", + "client_id", + b"client_id", + "keys", + b"keys", + "optional_values", + b"optional_values", + "values", + b"values", + "warnings", + b"warnings", + ], + ) -> None: ... + +global___ConfigResponse = ConfigResponse diff --git a/python/pyspark/sql/connect/proto/base_pb2_grpc.py b/python/pyspark/sql/connect/proto/base_pb2_grpc.py index aff5897f520f8..007e31fd0ea3b 100644 --- a/python/pyspark/sql/connect/proto/base_pb2_grpc.py +++ b/python/pyspark/sql/connect/proto/base_pb2_grpc.py @@ -40,6 +40,11 @@ def __init__(self, channel): request_serializer=spark_dot_connect_dot_base__pb2.AnalyzePlanRequest.SerializeToString, response_deserializer=spark_dot_connect_dot_base__pb2.AnalyzePlanResponse.FromString, ) + self.Config = channel.unary_unary( + "/spark.connect.SparkConnectService/Config", + request_serializer=spark_dot_connect_dot_base__pb2.ConfigRequest.SerializeToString, + response_deserializer=spark_dot_connect_dot_base__pb2.ConfigResponse.FromString, + ) class SparkConnectServiceServicer(object): @@ -60,6 +65,12 @@ def AnalyzePlan(self, request, context): context.set_details("Method not implemented!") raise NotImplementedError("Method not implemented!") + def Config(self, request, context): + """Update or fetch the configurations and returns a [[ConfigResponse]] containing the result.""" + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + def add_SparkConnectServiceServicer_to_server(servicer, server): rpc_method_handlers = { @@ -73,6 +84,11 @@ def add_SparkConnectServiceServicer_to_server(servicer, server): request_deserializer=spark_dot_connect_dot_base__pb2.AnalyzePlanRequest.FromString, response_serializer=spark_dot_connect_dot_base__pb2.AnalyzePlanResponse.SerializeToString, ), + "Config": grpc.unary_unary_rpc_method_handler( + servicer.Config, + request_deserializer=spark_dot_connect_dot_base__pb2.ConfigRequest.FromString, + response_serializer=spark_dot_connect_dot_base__pb2.ConfigResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( "spark.connect.SparkConnectService", rpc_method_handlers @@ -141,3 +157,32 @@ def AnalyzePlan( timeout, metadata, ) + + @staticmethod + def Config( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/spark.connect.SparkConnectService/Config", + spark_dot_connect_dot_base__pb2.ConfigRequest.SerializeToString, + spark_dot_connect_dot_base__pb2.ConfigResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d015e7df32b37..67a3f1b5fed07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -4177,7 +4177,12 @@ object SQLConf { * @param comment Additional info regarding to the removed config. For example, * reasons of config deprecation, what users should use instead of it. */ - case class DeprecatedConfig(key: String, version: String, comment: String) + case class DeprecatedConfig(key: String, version: String, comment: String) { + def toDeprecationString: String = { + s"The SQL config '$key' has been deprecated in Spark v$version " + + s"and may be removed in the future. $comment" + } + } /** * Maps deprecated SQL config keys to information about the deprecation. @@ -5148,11 +5153,8 @@ class SQLConf extends Serializable with Logging { * Logs a warning message if the given config key is deprecated. */ private def logDeprecationWarning(key: String): Unit = { - SQLConf.deprecatedSQLConfigs.get(key).foreach { - case DeprecatedConfig(configName, version, comment) => - logWarning( - s"The SQL config '$configName' has been deprecated in Spark v$version " + - s"and may be removed in the future. $comment") + SQLConf.deprecatedSQLConfigs.get(key).foreach { config => + logWarning(config.toDeprecationString) } } From 70b54dfd9c62ad05dd24c89b0fa079e78ad7838d Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 21 Feb 2023 16:48:37 -0800 Subject: [PATCH 02/14] Implement SparkSession.conf. --- .../main/protobuf/spark/connect/base.proto | 16 +-- .../service/SparkConnectConfigHandler.scala | 44 +++++--- dev/sparktestsupport/modules.py | 2 + python/pyspark/sql/conf.py | 4 +- python/pyspark/sql/connect/client.py | 100 ++++++++++++++++++ python/pyspark/sql/connect/conf.py | 99 +++++++++++++++++ python/pyspark/sql/connect/proto/base_pb2.py | 45 ++++---- python/pyspark/sql/connect/proto/base_pb2.pyi | 69 ++++++------ python/pyspark/sql/connect/session.py | 3 +- python/pyspark/sql/context.py | 4 +- .../sql/tests/connect/test_parity_conf.py | 36 +++++++ python/pyspark/sql/tests/test_conf.py | 6 +- 12 files changed, 350 insertions(+), 78 deletions(-) create mode 100644 python/pyspark/sql/connect/conf.py create mode 100644 python/pyspark/sql/tests/connect/test_parity_conf.py diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 522bc1113a945..840b2cfd641e9 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -183,6 +183,11 @@ message ExecutePlanResponse { } } +// The placeholder for the config request and response when the values can be optional. +message OptionalValue { + optional string value = 1; +} + // Request to update or fetch the configurations. message ConfigRequest { // (Required) @@ -209,7 +214,7 @@ message ConfigRequest { // Required when the 'operation' is 'SET'. // Optional when the 'operation' is 'GET', the values here will be used // as the default values. - repeated string values = 5; + repeated OptionalValue optional_values = 5; // (Optional) // @@ -229,6 +234,7 @@ message ConfigRequest { } } +// Response to the config request. message ConfigResponse { string client_id = 1; @@ -239,12 +245,14 @@ message ConfigResponse { // (Optional) // - // Available when the operation is 'GET', 'GET_ALL'. + // Available when the operation is 'GET_ALL'. + // Optional when the 'operation' is 'GET' without the default values. repeated string values = 3; // (Optional) // // Available when the operation is 'GET_OPTION'. + // Optional when the operation is 'GET' with the default values. repeated OptionalValue optional_values = 4; // (Optional) @@ -256,10 +264,6 @@ message ConfigResponse { // // Warning messages for deprecated or unsupported configurations. repeated string warnings = 6; - - message OptionalValue { - optional string value = 1; - } } // Main interface for the SparkConnect service. diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index 80f80d76e6539..c38fd4b769655 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -47,44 +47,40 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes request.getOperation match { case proto.ConfigRequest.Operation.OPERATION_SET => - if (request.getKeysCount != request.getValuesCount) { + if (request.getKeysCount != request.getOptionalValuesCount) { throw new UnsupportedOperationException("Keys and values should have the same length!") } request.getKeysList.asScala.iterator - .zip(request.getValuesList.asScala.iterator) - .foreach { case (key, value) => - conf.set(key, value) + .zip(request.getOptionalValuesList.asScala.iterator) + .foreach { case (key, optional_value) => + conf.set(key, SparkConnectConfigHandler.toOption(optional_value).orNull) getWarning(key).foreach(builder.addWarnings) } case proto.ConfigRequest.Operation.OPERATION_GET => - if (request.getValuesCount == 0) { + if (request.getOptionalValuesCount == 0) { request.getKeysList.asScala.iterator.foreach { key => builder.addValues(conf.get(key)) getWarning(key).foreach(builder.addWarnings) } } else { - if (request.getKeysCount != request.getValuesCount) { + if (request.getKeysCount != request.getOptionalValuesCount) { throw new UnsupportedOperationException( "Keys and values should have the same length!") } request.getKeysList.asScala.iterator - .zip(request.getValuesList.asScala.iterator) - .foreach { case (key, value) => - builder.addValues(conf.get(key, value)) + .zip(request.getOptionalValuesList.asScala.iterator) + .foreach { case (key, optional_value) => + builder.addOptionalValues(SparkConnectConfigHandler.toOptionalValue( + Option(conf.get(key, SparkConnectConfigHandler.toOption(optional_value).orNull)))) getWarning(key).foreach(builder.addWarnings) } } case proto.ConfigRequest.Operation.OPERATION_GET_OPTION => request.getKeysList.asScala.iterator.foreach { key => - conf.getOption(key) match { - case Some(value) => - builder.addOptionalValues( - proto.ConfigResponse.OptionalValue.newBuilder().setValue(value).build()) - case _ => - builder.addOptionalValues(proto.ConfigResponse.OptionalValue.newBuilder().build()) - } + builder.addOptionalValues( + SparkConnectConfigHandler.toOptionalValue(conf.getOption(key))) getWarning(key).foreach(builder.addWarnings) } @@ -139,4 +135,20 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes object SparkConnectConfigHandler { private[connect] val unsupportedConfigurations = Set("spark.sql.execution.arrow.enabled") + + def toOption(value: proto.OptionalValue): Option[String] = { + if (value.hasValue) { + Some(value.getValue) + } else { + None + } + } + + def toOptionalValue(value: Option[String]): proto.OptionalValue = { + val builder = proto.OptionalValue.newBuilder() + if (value.isDefined) { + builder.setValue(value.get) + } + builder.build() + } } diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 75a6b4401b86b..daecbc8485624 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -508,6 +508,7 @@ def __hash__(self): python_test_goals=[ # doctests "pyspark.sql.connect.catalog", + "pyspark.sql.connect.conf", "pyspark.sql.connect.group", "pyspark.sql.connect.session", "pyspark.sql.connect.window", @@ -523,6 +524,7 @@ def __hash__(self): "pyspark.sql.tests.connect.test_connect_column", "pyspark.sql.tests.connect.test_parity_datasources", "pyspark.sql.tests.connect.test_parity_catalog", + "pyspark.sql.tests.connect.test_parity_conf", "pyspark.sql.tests.connect.test_parity_serde", "pyspark.sql.tests.connect.test_parity_functions", "pyspark.sql.tests.connect.test_parity_group", diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index 40a36a26701a6..c14bc311a9fdf 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -40,7 +40,9 @@ def set(self, key: str, value: str) -> None: self._jconf.set(key, value) @since(2.0) - def get(self, key: str, default: Union[Optional[str], _NoValueType] = _NoValue) -> str: + def get( + self, key: str, default: Union[Optional[str], _NoValueType] = _NoValue + ) -> Optional[str]: """Returns the value of Spark runtime configuration property for the given key, assuming it is set. """ diff --git a/python/pyspark/sql/connect/client.py b/python/pyspark/sql/connect/client.py index 154dd161e9221..481cb41322f44 100644 --- a/python/pyspark/sql/connect/client.py +++ b/python/pyspark/sql/connect/client.py @@ -402,6 +402,34 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": ) +class ConfigResult: + def __init__( + self, + keys: List[str], + values: List[str], + optional_values: List[Optional[str]], + bools: List[bool], + warnings: List[str], + ): + self.keys = keys + self.values = values + self.optional_values = optional_values + self.bools = bools + self.warnings = warnings + + @classmethod + def fromProto(cls, pb: pb2.ConfigResponse) -> "ConfigResult": + return ConfigResult( + keys=list(pb.keys), + values=list(pb.values), + optional_values=[ + value.value if value.HasField("value") else None for value in pb.optional_values + ], + bools=list(pb.bools), + warnings=list(pb.warnings), + ) + + class SparkConnectClient(object): """ Conceptually the remote spark session that communicates with the server @@ -736,6 +764,78 @@ def _execute_and_fetch( metrics: List[PlanMetrics] = self._build_metrics(m) if m is not None else [] return table, metrics + def _config_request_with_metadata(self) -> pb2.ConfigRequest: + req = pb2.ConfigRequest() + req.client_id = self._session_id + if self._user_id: + req.user_context.user_id = self._user_id + return req + + def config( + self, + operation: str, + keys: Optional[List[str]], + optional_values: Optional[List[Optional[str]]] = None, + prefix: Optional[str] = None, + ) -> ConfigResult: + """ + Call the config RPC of Spark Connect. + + Parameters + ---------- + operation : str + Operation kind + + Returns + ------- + The result of the config call. + """ + req = self._config_request_with_metadata() + if operation == "set": + req.operation = req.Operation.OPERATION_SET + elif operation == "get": + req.operation = req.Operation.OPERATION_GET + elif operation == "get_option": + req.operation = req.Operation.OPERATION_GET_OPTION + elif operation == "get_all": + req.operation = req.Operation.OPERATION_GET_ALL + elif operation == "unset": + req.operation = req.Operation.OPERATION_UNSET + elif operation == "contains": + req.operation = req.Operation.OPERATION_CONTAINS + elif operation == "is_modifiable": + req.operation = req.Operation.OPERATION_IS_MODIFIABLE + else: + raise ValueError( + f"Unknown operation: {operation}. Accepted operations are " + "'set', 'get', 'get_option', 'get_all', 'unset', 'contains', 'is_modifiable'." + ) + if keys is not None: + req.keys.extend(keys) + if optional_values is not None: + for value in optional_values: + optional_value = pb2.OptionalValue() + if value is not None: + optional_value.value = value + req.optional_values.append(optional_value) + if prefix is not None: + req.prefix = prefix + try: + for attempt in Retrying( + can_retry=SparkConnectClient.retry_exception, **self._retry_policy + ): + with attempt: + resp = self._stub.Config(req, metadata=self._builder.metadata()) + if resp.client_id != self._session_id: + raise SparkConnectException( + "Received incorrect session identifier for request:" + f"{resp.client_id} != {self._session_id}" + ) + return ConfigResult.fromProto(resp) + raise SparkConnectException("Invalid state during retry exception handling.") + except grpc.RpcError as rpc_error: + self._handle_error(rpc_error) + def _handle_error(self, rpc_error: grpc.RpcError) -> NoReturn: """ Error handling helper for dealing with GRPC Errors. On the server side, certain diff --git a/python/pyspark/sql/connect/conf.py b/python/pyspark/sql/connect/conf.py new file mode 100644 index 0000000000000..c5e6eeaf9a98a --- /dev/null +++ b/python/pyspark/sql/connect/conf.py @@ -0,0 +1,99 @@ +# +# 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. +# +from typing import Any, Optional, Union, cast + +from pyspark import _NoValue +from pyspark._globals import _NoValueType +from pyspark.sql.conf import RuntimeConfig as PySparkRuntimeConfig +from pyspark.sql.connect.client import SparkConnectClient + + +class RuntimeConf: + def __init__(self, client: SparkConnectClient) -> None: + """Create a new RuntimeConfig.""" + self._client = client + + __init__.__doc__ = PySparkRuntimeConfig.__init__.__doc__ + + def set(self, key: str, value: str) -> None: + self._client.config("set", keys=[key], optional_values=[value]) + + set.__doc__ = PySparkRuntimeConfig.set.__doc__ + + def get( + self, key: str, default: Union[Optional[str], _NoValueType] = _NoValue + ) -> Optional[str]: + self._checkType(key, "key") + if default is _NoValue: + return self._client.config("get", keys=[key]).values[0] + else: + if default is not None: + self._checkType(default, "default") + return self._client.config( + "get", keys=[key], optional_values=[cast(Optional[str], default)] + ).optional_values[0] + + get.__doc__ = PySparkRuntimeConfig.get.__doc__ + + def unset(self, key: str) -> None: + self._client.config("unset", keys=[key]) + + unset.__doc__ = PySparkRuntimeConfig.unset.__doc__ + + def isModifiable(self, key: str) -> bool: + return self._client.config("is_modifiable", keys=[key]).bools[0] + + isModifiable.__doc__ = PySparkRuntimeConfig.isModifiable.__doc__ + + def _checkType(self, obj: Any, identifier: str) -> None: + """Assert that an object is of type str.""" + if not isinstance(obj, str): + raise TypeError( + "expected %s '%s' to be a string (was '%s')" % (identifier, obj, type(obj).__name__) + ) + + +RuntimeConf.__doc__ = PySparkRuntimeConfig.__doc__ + + +def _test() -> None: + import sys + import doctest + from pyspark.sql import SparkSession as PySparkSession + import pyspark.sql.connect.conf + + globs = pyspark.sql.connect.conf.__dict__.copy() + globs["spark"] = ( + PySparkSession.builder.appName("sql.connect.conf tests").remote("local[4]").getOrCreate() + ) + + (failure_count, test_count) = doctest.testmod( + pyspark.sql.connect.conf, + globs=globs, + optionflags=doctest.ELLIPSIS + | doctest.NORMALIZE_WHITESPACE + | doctest.IGNORE_EXCEPTION_DETAIL, + ) + + globs["spark"].stop() + + if failure_count: + sys.exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 4ee8a5b84243e..56843821c5ef8 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"\xcf\x03\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0e\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12\x12\n\x04keys\x18\x04 \x03(\tR\x04keys\x12\x16\n\x06values\x18\x05 \x03(\tR\x06values\x12\x1b\n\x06prefix\x18\x06 \x01(\tH\x00R\x06prefix\x88\x01\x01"\xc7\x01\n\tOperation\x12\x19\n\x15OPERATION_UNSPECIFIED\x10\x00\x12\x11\n\rOPERATION_SET\x10\x01\x12\x11\n\rOPERATION_GET\x10\x02\x12\x18\n\x14OPERATION_GET_OPTION\x10\x03\x12\x15\n\x11OPERATION_GET_ALL\x10\x04\x12\x13\n\x0fOPERATION_UNSET\x10\x05\x12\x16\n\x12OPERATION_CONTAINS\x10\x06\x12\x1b\n\x17OPERATION_IS_MODIFIABLE\x10\x07\x42\t\n\x07_prefix"\x97\x02\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x12\n\x04keys\x18\x02 \x03(\tR\x04keys\x12\x16\n\x06values\x18\x03 \x03(\tR\x06values\x12T\n\x0foptional_values\x18\x04 \x03(\x0b\x32+.spark.connect.ConfigResponse.OptionalValueR\x0eoptionalValues\x12\x14\n\x05\x62ools\x18\x05 \x03(\x08R\x05\x62ools\x12\x1a\n\x08warnings\x18\x06 \x03(\tR\x08warnings\x1a\x34\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xfe\x03\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0e\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12\x12\n\x04keys\x18\x04 \x03(\tR\x04keys\x12\x45\n\x0foptional_values\x18\x05 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x0eoptionalValues\x12\x1b\n\x06prefix\x18\x06 \x01(\tH\x00R\x06prefix\x88\x01\x01"\xc7\x01\n\tOperation\x12\x19\n\x15OPERATION_UNSPECIFIED\x10\x00\x12\x11\n\rOPERATION_SET\x10\x01\x12\x11\n\rOPERATION_GET\x10\x02\x12\x18\n\x14OPERATION_GET_OPTION\x10\x03\x12\x15\n\x11OPERATION_GET_ALL\x10\x04\x12\x13\n\x0fOPERATION_UNSET\x10\x05\x12\x16\n\x12OPERATION_CONTAINS\x10\x06\x12\x1b\n\x17OPERATION_IS_MODIFIABLE\x10\x07\x42\t\n\x07_prefix"\xd2\x01\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x12\n\x04keys\x18\x02 \x03(\tR\x04keys\x12\x16\n\x06values\x18\x03 \x03(\tR\x06values\x12\x45\n\x0foptional_values\x18\x04 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x0eoptionalValues\x12\x14\n\x05\x62ools\x18\x05 \x03(\x08R\x05\x62ools\x12\x1a\n\x08warnings\x18\x06 \x03(\tR\x08warnings2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -58,9 +58,9 @@ _EXECUTEPLANRESPONSE_METRICS_METRICVALUE = _EXECUTEPLANRESPONSE_METRICS.nested_types_by_name[ "MetricValue" ] +_OPTIONALVALUE = DESCRIPTOR.message_types_by_name["OptionalValue"] _CONFIGREQUEST = DESCRIPTOR.message_types_by_name["ConfigRequest"] _CONFIGRESPONSE = DESCRIPTOR.message_types_by_name["ConfigResponse"] -_CONFIGRESPONSE_OPTIONALVALUE = _CONFIGRESPONSE.nested_types_by_name["OptionalValue"] _EXPLAIN_EXPLAINMODE = _EXPLAIN.enum_types_by_name["ExplainMode"] _CONFIGREQUEST_OPERATION = _CONFIGREQUEST.enum_types_by_name["Operation"] Plan = _reflection.GeneratedProtocolMessageType( @@ -190,6 +190,17 @@ _sym_db.RegisterMessage(ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntry) _sym_db.RegisterMessage(ExecutePlanResponse.Metrics.MetricValue) +OptionalValue = _reflection.GeneratedProtocolMessageType( + "OptionalValue", + (_message.Message,), + { + "DESCRIPTOR": _OPTIONALVALUE, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.OptionalValue) + }, +) +_sym_db.RegisterMessage(OptionalValue) + ConfigRequest = _reflection.GeneratedProtocolMessageType( "ConfigRequest", (_message.Message,), @@ -205,22 +216,12 @@ "ConfigResponse", (_message.Message,), { - "OptionalValue": _reflection.GeneratedProtocolMessageType( - "OptionalValue", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_OPTIONALVALUE, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.OptionalValue) - }, - ), "DESCRIPTOR": _CONFIGRESPONSE, "__module__": "spark.connect.base_pb2" # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse) }, ) _sym_db.RegisterMessage(ConfigResponse) -_sym_db.RegisterMessage(ConfigResponse.OptionalValue) _SPARKCONNECTSERVICE = DESCRIPTOR.services_by_name["SparkConnectService"] if _descriptor._USE_C_DESCRIPTORS == False: @@ -255,14 +256,14 @@ _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 2017 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 2019 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 2107 - _CONFIGREQUEST._serialized_start = 2110 - _CONFIGREQUEST._serialized_end = 2573 - _CONFIGREQUEST_OPERATION._serialized_start = 2363 - _CONFIGREQUEST_OPERATION._serialized_end = 2562 - _CONFIGRESPONSE._serialized_start = 2576 - _CONFIGRESPONSE._serialized_end = 2855 - _CONFIGRESPONSE_OPTIONALVALUE._serialized_start = 2803 - _CONFIGRESPONSE_OPTIONALVALUE._serialized_end = 2855 - _SPARKCONNECTSERVICE._serialized_start = 2858 - _SPARKCONNECTSERVICE._serialized_end = 3130 + _OPTIONALVALUE._serialized_start = 2109 + _OPTIONALVALUE._serialized_end = 2161 + _CONFIGREQUEST._serialized_start = 2164 + _CONFIGREQUEST._serialized_end = 2674 + _CONFIGREQUEST_OPERATION._serialized_start = 2464 + _CONFIGREQUEST_OPERATION._serialized_end = 2663 + _CONFIGRESPONSE._serialized_start = 2677 + _CONFIGRESPONSE._serialized_end = 2887 + _SPARKCONNECTSERVICE._serialized_start = 2890 + _SPARKCONNECTSERVICE._serialized_end = 3162 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 21feba0e88f48..a72d2e79bec22 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -571,6 +571,30 @@ class ExecutePlanResponse(google.protobuf.message.Message): global___ExecutePlanResponse = ExecutePlanResponse +class OptionalValue(google.protobuf.message.Message): + """The placeholder for the config request and response when the values can be optional.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUE_FIELD_NUMBER: builtins.int + value: builtins.str + def __init__( + self, + *, + value: builtins.str | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["_value", b"_value", "value", b"value"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["_value", b"_value", "value", b"value"] + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_value", b"_value"] + ) -> typing_extensions.Literal["value"] | None: ... + +global___OptionalValue = OptionalValue + class ConfigRequest(google.protobuf.message.Message): """Request to update or fetch the configurations.""" @@ -610,7 +634,7 @@ class ConfigRequest(google.protobuf.message.Message): USER_CONTEXT_FIELD_NUMBER: builtins.int OPERATION_FIELD_NUMBER: builtins.int KEYS_FIELD_NUMBER: builtins.int - VALUES_FIELD_NUMBER: builtins.int + OPTIONAL_VALUES_FIELD_NUMBER: builtins.int PREFIX_FIELD_NUMBER: builtins.int client_id: builtins.str """(Required) @@ -633,9 +657,11 @@ class ConfigRequest(google.protobuf.message.Message): 'CONTAINS', 'IS_MODIFIABLE'. """ @property - def values( + def optional_values( self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___OptionalValue + ]: """(Optional) Corresponding values to the keys. @@ -656,7 +682,7 @@ class ConfigRequest(google.protobuf.message.Message): user_context: global___UserContext | None = ..., operation: global___ConfigRequest.Operation.ValueType = ..., keys: collections.abc.Iterable[builtins.str] | None = ..., - values: collections.abc.Iterable[builtins.str] | None = ..., + optional_values: collections.abc.Iterable[global___OptionalValue] | None = ..., prefix: builtins.str | None = ..., ) -> None: ... def HasField( @@ -676,12 +702,12 @@ class ConfigRequest(google.protobuf.message.Message): b"keys", "operation", b"operation", + "optional_values", + b"optional_values", "prefix", b"prefix", "user_context", b"user_context", - "values", - b"values", ], ) -> None: ... def WhichOneof( @@ -691,27 +717,9 @@ class ConfigRequest(google.protobuf.message.Message): global___ConfigRequest = ConfigRequest class ConfigResponse(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - class OptionalValue(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor + """Response to the config request.""" - VALUE_FIELD_NUMBER: builtins.int - value: builtins.str - def __init__( - self, - *, - value: builtins.str | None = ..., - ) -> None: ... - def HasField( - self, field_name: typing_extensions.Literal["_value", b"_value", "value", b"value"] - ) -> builtins.bool: ... - def ClearField( - self, field_name: typing_extensions.Literal["_value", b"_value", "value", b"value"] - ) -> None: ... - def WhichOneof( - self, oneof_group: typing_extensions.Literal["_value", b"_value"] - ) -> typing_extensions.Literal["value"] | None: ... + DESCRIPTOR: google.protobuf.descriptor.Descriptor CLIENT_ID_FIELD_NUMBER: builtins.int KEYS_FIELD_NUMBER: builtins.int @@ -734,17 +742,19 @@ class ConfigResponse(google.protobuf.message.Message): ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: """(Optional) - Available when the operation is 'GET', 'GET_ALL'. + Available when the operation is 'GET_ALL'. + Optional when the 'operation' is 'GET' without the default values. """ @property def optional_values( self, ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ - global___ConfigResponse.OptionalValue + global___OptionalValue ]: """(Optional) Available when the operation is 'GET_OPTION'. + Optional when the operation is 'GET' with the default values. """ @property def bools( @@ -768,8 +778,7 @@ class ConfigResponse(google.protobuf.message.Message): client_id: builtins.str = ..., keys: collections.abc.Iterable[builtins.str] | None = ..., values: collections.abc.Iterable[builtins.str] | None = ..., - optional_values: collections.abc.Iterable[global___ConfigResponse.OptionalValue] - | None = ..., + optional_values: collections.abc.Iterable[global___OptionalValue] | None = ..., bools: collections.abc.Iterable[builtins.bool] | None = ..., warnings: collections.abc.Iterable[builtins.str] | None = ..., ) -> None: ... diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 08e63f544e23d..5c685df91df6e 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -47,6 +47,7 @@ from pyspark import SparkContext, SparkConf, __version__ from pyspark.sql.connect.client import SparkConnectClient +from pyspark.sql.connect.conf import RuntimeConf from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.connect.plan import SQL, Range, LocalRelation from pyspark.sql.connect.readwriter import DataFrameReader @@ -422,7 +423,7 @@ def newSession(self) -> Any: @property def conf(self) -> Any: - raise NotImplementedError("conf() is not implemented.") + return RuntimeConf(self.client) @property def sparkContext(self) -> Any: diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 3c47ebfb97365..480013222b73d 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -193,7 +193,9 @@ def setConf(self, key: str, value: Union[bool, int, str]) -> None: """ self.sparkSession.conf.set(key, value) # type: ignore[arg-type] - def getConf(self, key: str, defaultValue: Union[Optional[str], _NoValueType] = _NoValue) -> str: + def getConf( + self, key: str, defaultValue: Union[Optional[str], _NoValueType] = _NoValue + ) -> Optional[str]: """Returns the value of Spark SQL configuration property for the given key. If the key is not set and defaultValue is set, return diff --git a/python/pyspark/sql/tests/connect/test_parity_conf.py b/python/pyspark/sql/tests/connect/test_parity_conf.py new file mode 100644 index 0000000000000..554f05f27ea77 --- /dev/null +++ b/python/pyspark/sql/tests/connect/test_parity_conf.py @@ -0,0 +1,36 @@ +# +# 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. +# + +from pyspark.sql.tests.test_conf import ConfTestsMixin +from pyspark.testing.connectutils import ReusedConnectTestCase + + +class ConfParityTests(ConfTestsMixin, ReusedConnectTestCase): + pass + + +if __name__ == "__main__": + import unittest + from pyspark.sql.tests.connect.test_parity_conf import * # noqa: F401 + + try: + import xmlrunner # type: ignore[import] + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/test_conf.py b/python/pyspark/sql/tests/test_conf.py index a8fa59c036408..7a58273b34e0b 100644 --- a/python/pyspark/sql/tests/test_conf.py +++ b/python/pyspark/sql/tests/test_conf.py @@ -18,7 +18,7 @@ from pyspark.testing.sqlutils import ReusedSQLTestCase -class ConfTests(ReusedSQLTestCase): +class ConfTestsMixin: def test_conf(self): spark = self.spark spark.conf.set("bogo", "sipeo") @@ -43,6 +43,10 @@ def test_conf(self): self.assertEqual(spark.conf.get("spark.sql.sources.partitionOverwriteMode", None), None) +class ConfTests(ConfTestsMixin, ReusedSQLTestCase): + pass + + if __name__ == "__main__": import unittest from pyspark.sql.tests.test_conf import * # noqa: F401 From 9d199838411d82fd70350d303aaff5af10806013 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 22 Feb 2023 15:52:13 -0800 Subject: [PATCH 03/14] Fix. --- python/pyspark/pandas/utils.py | 2 +- python/pyspark/sql/conf.py | 2 +- python/pyspark/sql/connect/conf.py | 6 +++++- python/pyspark/sql/context.py | 2 +- python/pyspark/sql/tests/test_conf.py | 18 ++++++++++++++++++ 5 files changed, 26 insertions(+), 4 deletions(-) diff --git a/python/pyspark/pandas/utils.py b/python/pyspark/pandas/utils.py index 9deb0147e6644..c48dc8449cd75 100644 --- a/python/pyspark/pandas/utils.py +++ b/python/pyspark/pandas/utils.py @@ -473,7 +473,7 @@ def default_session() -> SparkSession: # Turn ANSI off when testing the pandas API on Spark since # the behavior of pandas API on Spark follows pandas, not SQL. if is_testing(): - spark.conf.set("spark.sql.ansi.enabled", False) # type: ignore[arg-type] + spark.conf.set("spark.sql.ansi.enabled", False) if spark.conf.get("spark.sql.ansi.enabled") == "true": log_advice( "The config 'spark.sql.ansi.enabled' is set to True. " diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index c14bc311a9fdf..01e77b10fc6ae 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -35,7 +35,7 @@ def __init__(self, jconf: JavaObject) -> None: self._jconf = jconf @since(2.0) - def set(self, key: str, value: str) -> None: + def set(self, key: str, value: Union[str, int, bool]) -> None: """Sets the given Spark runtime configuration property.""" self._jconf.set(key, value) diff --git a/python/pyspark/sql/connect/conf.py b/python/pyspark/sql/connect/conf.py index c5e6eeaf9a98a..d2765b568293d 100644 --- a/python/pyspark/sql/connect/conf.py +++ b/python/pyspark/sql/connect/conf.py @@ -29,7 +29,11 @@ def __init__(self, client: SparkConnectClient) -> None: __init__.__doc__ = PySparkRuntimeConfig.__init__.__doc__ - def set(self, key: str, value: str) -> None: + def set(self, key: str, value: Union[str, int, bool]) -> None: + if isinstance(value, bool): + value = "true" if value else "false" + elif isinstance(value, int): + value = str(value) self._client.config("set", keys=[key], optional_values=[value]) set.__doc__ = PySparkRuntimeConfig.set.__doc__ diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 480013222b73d..99f97977ccc7c 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -191,7 +191,7 @@ def setConf(self, key: str, value: Union[bool, int, str]) -> None: .. versionadded:: 1.3.0 """ - self.sparkSession.conf.set(key, value) # type: ignore[arg-type] + self.sparkSession.conf.set(key, value) def getConf( self, key: str, defaultValue: Union[Optional[str], _NoValueType] = _NoValue diff --git a/python/pyspark/sql/tests/test_conf.py b/python/pyspark/sql/tests/test_conf.py index 7a58273b34e0b..499db606ed9d8 100644 --- a/python/pyspark/sql/tests/test_conf.py +++ b/python/pyspark/sql/tests/test_conf.py @@ -14,7 +14,9 @@ # See the License for the specific language governing permissions and # limitations under the License. # +from decimal import Decimal +from pyspark.errors import IllegalArgumentException from pyspark.testing.sqlutils import ReusedSQLTestCase @@ -42,6 +44,22 @@ def test_conf(self): # `defaultValue` in `spark.conf.get` is set to None. self.assertEqual(spark.conf.get("spark.sql.sources.partitionOverwriteMode", None), None) + def test_conf_with_python_objects(self): + spark = self.spark + + for value, expected in [(True, "true"), (False, "false")]: + spark.conf.set("foo", value) + self.assertEqual(spark.conf.get("foo"), expected) + + spark.conf.set("foo", 1) + self.assertEqual(spark.conf.get("foo"), "1") + + with self.assertRaises(IllegalArgumentException): + spark.conf.set("foo", None) + + with self.assertRaises(Exception): + spark.conf.set("foo", Decimal(1)) + class ConfTests(ConfTestsMixin, ReusedSQLTestCase): pass From adbcc57036918426f9c4967a43f0807e3faa4d94 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 22 Feb 2023 15:59:58 -0800 Subject: [PATCH 04/14] warnings --- python/pyspark/sql/connect/conf.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/connect/conf.py b/python/pyspark/sql/connect/conf.py index d2765b568293d..9f389477474cb 100644 --- a/python/pyspark/sql/connect/conf.py +++ b/python/pyspark/sql/connect/conf.py @@ -15,6 +15,7 @@ # limitations under the License. # from typing import Any, Optional, Union, cast +import warnings from pyspark import _NoValue from pyspark._globals import _NoValueType @@ -34,7 +35,9 @@ def set(self, key: str, value: Union[str, int, bool]) -> None: value = "true" if value else "false" elif isinstance(value, int): value = str(value) - self._client.config("set", keys=[key], optional_values=[value]) + result = self._client.config("set", keys=[key], optional_values=[value]) + for warn in result.warnings: + warnings.warn(warn) set.__doc__ = PySparkRuntimeConfig.set.__doc__ @@ -54,7 +57,9 @@ def get( get.__doc__ = PySparkRuntimeConfig.get.__doc__ def unset(self, key: str) -> None: - self._client.config("unset", keys=[key]) + result = self._client.config("unset", keys=[key]) + for warn in result.warnings: + warnings.warn(warn) unset.__doc__ = PySparkRuntimeConfig.unset.__doc__ From 33b61e0540b7df8272700717899afb103302a0b5 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Feb 2023 13:14:28 -0800 Subject: [PATCH 05/14] Fix. --- .../main/protobuf/spark/connect/base.proto | 90 +++-- .../service/SparkConnectConfigHandler.scala | 192 +++++---- python/pyspark/sql/connect/client.py | 38 +- python/pyspark/sql/connect/conf.py | 40 +- python/pyspark/sql/connect/proto/base_pb2.py | 162 +++++++- python/pyspark/sql/connect/proto/base_pb2.pyi | 364 ++++++++++++++---- 6 files changed, 646 insertions(+), 240 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 840b2cfd641e9..8bf705460782c 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -188,6 +188,18 @@ message OptionalValue { optional string value = 1; } +// The key-value pair for the config request and response. +message KeyValue { + string key = 1; + string value = 2; +} + +// The key-value pair for the config request and response when the value can be optional. +message OptionalKeyValue { + string key = 1; + OptionalValue value = 2; +} + // Request to update or fetch the configurations. message ConfigRequest { // (Required) @@ -199,38 +211,60 @@ message ConfigRequest { // (Required) User context UserContext user_context = 2; + // (Required) The operation for the config. Operation operation = 3; - // (Optional) - // - // Identify which keys will be updated or fetched. - // Required when the 'operation' is 'SET', 'GET', 'GET_OPTION', 'UNSET', - // 'CONTAINS', 'IS_MODIFIABLE'. - repeated string keys = 4; + message Operation { + oneof op_type { + Set set = 1; + Get get = 2; + GetWithDefault get_with_default = 3; + GetOption get_option = 4; + GetAll get_all = 5; + Unset unset = 6; + Contains contains = 7; + IsModifiable is_modifiable = 8; + } + } - // (Optional) - // - // Corresponding values to the keys. - // Required when the 'operation' is 'SET'. - // Optional when the 'operation' is 'GET', the values here will be used - // as the default values. - repeated OptionalValue optional_values = 5; + message Set { + // (Required) The config key-value pairs to set. + repeated KeyValue pairs = 1; + } - // (Optional) - // - // Only used when the 'operation' is 'GET_ALL'. - // If prefix is given, only parameters that start with the prefix will be returned. - optional string prefix = 6; - - enum Operation { - OPERATION_UNSPECIFIED = 0; - OPERATION_SET = 1; - OPERATION_GET = 2; - OPERATION_GET_OPTION = 3; - OPERATION_GET_ALL = 4; - OPERATION_UNSET = 5; - OPERATION_CONTAINS = 6; - OPERATION_IS_MODIFIABLE = 7; + message Get { + // (Required) The config keys to get. + repeated string keys = 1; + } + + message GetWithDefault { + // (Required) The config key-value paris to get. The value will be used as the default value. + repeated OptionalKeyValue pairs = 1; + } + + message GetOption { + // (Required) The config keys to get optionally. + repeated string keys = 1; + } + + message GetAll { + // (Optional) The prefix of the config key to get. + optional string prefix = 1; + } + + message Unset { + // (Required) The config keys to unset. + repeated string keys = 1; + } + + message Contains { + // (Required) The config keys to check if the config contains them. + repeated string keys = 1; + } + + message IsModifiable { + // (Required) The config keys to check the config is modifiable. + repeated string keys = 1; } } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index c38fd4b769655..d81e6be9789a8 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -23,7 +23,7 @@ import io.grpc.stub.StreamObserver import org.apache.spark.connect.proto import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.internal.SQLConf class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigResponse]) @@ -35,91 +35,129 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes .getOrCreateIsolatedSession(request.getUserContext.getUserId, request.getClientId) .session - val builder = handleSQLConf(request, session) + val builder = request.getOperation.getOpTypeCase match { + case proto.ConfigRequest.Operation.OpTypeCase.SET => + handleSet(request.getOperation.getSet, session.conf) + case proto.ConfigRequest.Operation.OpTypeCase.GET => + handleGet(request.getOperation.getGet, session.conf) + case proto.ConfigRequest.Operation.OpTypeCase.GET_WITH_DEFAULT => + handleGetWithDefault(request.getOperation.getGetWithDefault, session.conf) + case proto.ConfigRequest.Operation.OpTypeCase.GET_OPTION => + handleGetOption(request.getOperation.getGetOption, session.conf) + case proto.ConfigRequest.Operation.OpTypeCase.GET_ALL => + handleGetAll(request.getOperation.getGetAll, session.conf) + case proto.ConfigRequest.Operation.OpTypeCase.UNSET => + handleUnset(request.getOperation.getUnset, session.conf) + case proto.ConfigRequest.Operation.OpTypeCase.CONTAINS => + handleContains(request.getOperation.getContains, session.conf) + case proto.ConfigRequest.Operation.OpTypeCase.IS_MODIFIABLE => + handleIsModifiable(request.getOperation.getIsModifiable, session.conf) + case _ => throw new UnsupportedOperationException(s"${request.getOperation} not supported.") + } + builder.setClientId(request.getClientId) responseObserver.onNext(builder.build()) responseObserver.onCompleted() } - private def handleSQLConf(request: proto.ConfigRequest, session: SparkSession) = { - val conf = session.conf + private def handleSet( + operation: proto.ConfigRequest.Set, + conf: RuntimeConfig): proto.ConfigResponse.Builder = { val builder = proto.ConfigResponse.newBuilder() + operation.getPairsList.asScala.iterator.foreach { pair => + val key = pair.getKey + val value = pair.getValue + conf.set(key, value) + getWarning(key).foreach(builder.addWarnings) + } + builder + } - request.getOperation match { - case proto.ConfigRequest.Operation.OPERATION_SET => - if (request.getKeysCount != request.getOptionalValuesCount) { - throw new UnsupportedOperationException("Keys and values should have the same length!") - } - request.getKeysList.asScala.iterator - .zip(request.getOptionalValuesList.asScala.iterator) - .foreach { case (key, optional_value) => - conf.set(key, SparkConnectConfigHandler.toOption(optional_value).orNull) - getWarning(key).foreach(builder.addWarnings) - } - - case proto.ConfigRequest.Operation.OPERATION_GET => - if (request.getOptionalValuesCount == 0) { - request.getKeysList.asScala.iterator.foreach { key => - builder.addValues(conf.get(key)) - getWarning(key).foreach(builder.addWarnings) - } - } else { - if (request.getKeysCount != request.getOptionalValuesCount) { - throw new UnsupportedOperationException( - "Keys and values should have the same length!") - } - request.getKeysList.asScala.iterator - .zip(request.getOptionalValuesList.asScala.iterator) - .foreach { case (key, optional_value) => - builder.addOptionalValues(SparkConnectConfigHandler.toOptionalValue( - Option(conf.get(key, SparkConnectConfigHandler.toOption(optional_value).orNull)))) - getWarning(key).foreach(builder.addWarnings) - } - } - - case proto.ConfigRequest.Operation.OPERATION_GET_OPTION => - request.getKeysList.asScala.iterator.foreach { key => - builder.addOptionalValues( - SparkConnectConfigHandler.toOptionalValue(conf.getOption(key))) - getWarning(key).foreach(builder.addWarnings) - } - - case proto.ConfigRequest.Operation.OPERATION_GET_ALL => - val results = if (request.hasPrefix) { - val prefix = request.getPrefix - conf.getAll.iterator - .filter { case (key, value) => key.startsWith(prefix) } - .map { case (key, value) => (key.substring(prefix.length), value) } - } else { - conf.getAll.iterator - } - results.foreach { case (key, value) => - builder.addKeys(key).addValues(value) - getWarning(key).foreach(builder.addWarnings) - } - - case proto.ConfigRequest.Operation.OPERATION_UNSET => - request.getKeysList.asScala.iterator.foreach { key => - conf.unset(key) - getWarning(key).foreach(builder.addWarnings) - } - - case proto.ConfigRequest.Operation.OPERATION_CONTAINS => - request.getKeysList.asScala.iterator.foreach { key => - builder.addBools(conf.contains(key)) - getWarning(key).foreach(builder.addWarnings) - } - - case proto.ConfigRequest.Operation.OPERATION_IS_MODIFIABLE => - request.getKeysList.asScala.iterator.foreach { key => - builder.addBools(conf.isModifiable(key)) - getWarning(key).foreach(builder.addWarnings) - } - - case other => - throw new UnsupportedOperationException(s"Unsupported operation $other in SQLConf.") + private def handleGet( + operation: proto.ConfigRequest.Get, + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => + builder.addValues(conf.get(key)) + getWarning(key).foreach(builder.addWarnings) } + builder + } + private def handleGetWithDefault( + operation: proto.ConfigRequest.GetWithDefault, + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getPairsList.asScala.iterator.foreach { pair => + val key = pair.getKey + val value = SparkConnectConfigHandler.toOption(pair.getValue).orNull + builder.addOptionalValues( + SparkConnectConfigHandler.toOptionalValue(Option(conf.get(key, value)))) + getWarning(key).foreach(builder.addWarnings) + } + builder + } + + private def handleGetOption( + operation: proto.ConfigRequest.GetOption, + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => + builder.addOptionalValues(SparkConnectConfigHandler.toOptionalValue(conf.getOption(key))) + getWarning(key).foreach(builder.addWarnings) + } + builder + } + + private def handleGetAll( + operation: proto.ConfigRequest.GetAll, + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + val results = if (operation.hasPrefix) { + val prefix = operation.getPrefix + conf.getAll.iterator + .filter { case (key, _) => key.startsWith(prefix) } + .map { case (key, value) => (key.substring(prefix.length), value) } + } else { + conf.getAll.iterator + } + results.foreach { case (key, value) => + builder.addKeys(key).addValues(value) + getWarning(key).foreach(builder.addWarnings) + } + builder + } + + private def handleUnset( + operation: proto.ConfigRequest.Unset, + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => + conf.unset(key) + getWarning(key).foreach(builder.addWarnings) + } + builder + } + + private def handleContains( + operation: proto.ConfigRequest.Contains, + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => + builder.addBools(conf.contains(key)) + getWarning(key).foreach(builder.addWarnings) + } + builder + } + + private def handleIsModifiable( + operation: proto.ConfigRequest.IsModifiable, + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => + builder.addBools(conf.isModifiable(key)) + getWarning(key).foreach(builder.addWarnings) + } builder } diff --git a/python/pyspark/sql/connect/client.py b/python/pyspark/sql/connect/client.py index 481cb41322f44..1eaf46541bf45 100644 --- a/python/pyspark/sql/connect/client.py +++ b/python/pyspark/sql/connect/client.py @@ -771,13 +771,7 @@ def _config_request_with_metadata(self) -> pb2.ConfigRequest: req.user_context.user_id = self._user_id return req - def config( - self, - operation: str, - keys: Optional[List[str]], - optional_values: Optional[List[Optional[str]]] = None, - prefix: Optional[str] = None, - ) -> ConfigResult: + def config(self, operation: pb2.ConfigRequest.Operation) -> ConfigResult: """ Call the config RPC of Spark Connect. @@ -791,35 +785,7 @@ def config( The result of the config call. """ req = self._config_request_with_metadata() - if operation == "set": - req.operation = req.Operation.OPERATION_SET - elif operation == "get": - req.operation = req.Operation.OPERATION_GET - elif operation == "get_option": - req.operation = req.Operation.OPERATION_GET_OPTION - elif operation == "get_all": - req.operation = req.Operation.OPERATION_GET_ALL - elif operation == "unset": - req.operation = req.Operation.OPERATION_UNSET - elif operation == "contains": - req.operation = req.Operation.OPERATION_CONTAINS - elif operation == "is_modifiable": - req.operation = req.Operation.OPERATION_IS_MODIFIABLE - else: - raise ValueError( - f"Unknown operation: {operation}. Accepted operations are " - "'set', 'get', 'get_option', 'get_all', 'unset', 'contains', 'is_modifiable'." - ) - if keys is not None: - req.keys.extend(keys) - if optional_values is not None: - for value in optional_values: - optional_value = pb2.OptionalValue() - if value is not None: - optional_value.value = value - req.optional_values.append(optional_value) - if prefix is not None: - req.prefix = prefix + req.operation.CopyFrom(operation) try: for attempt in Retrying( can_retry=SparkConnectClient.retry_exception, **self._retry_policy diff --git a/python/pyspark/sql/connect/conf.py b/python/pyspark/sql/connect/conf.py index 9f389477474cb..adcb7197d48d8 100644 --- a/python/pyspark/sql/connect/conf.py +++ b/python/pyspark/sql/connect/conf.py @@ -19,7 +19,9 @@ from pyspark import _NoValue from pyspark._globals import _NoValueType +from pyspark.errors import IllegalArgumentException from pyspark.sql.conf import RuntimeConfig as PySparkRuntimeConfig +from pyspark.sql.connect import proto from pyspark.sql.connect.client import SparkConnectClient @@ -35,7 +37,9 @@ def set(self, key: str, value: Union[str, int, bool]) -> None: value = "true" if value else "false" elif isinstance(value, int): value = str(value) - result = self._client.config("set", keys=[key], optional_values=[value]) + op_set = proto.ConfigRequest.Set(pairs=[_key_value(key, value)]) + operation = proto.ConfigRequest.Operation(set=op_set) + result = self._client.config(operation) for warn in result.warnings: warnings.warn(warn) @@ -46,25 +50,33 @@ def get( ) -> Optional[str]: self._checkType(key, "key") if default is _NoValue: - return self._client.config("get", keys=[key]).values[0] + op_get = proto.ConfigRequest.Get(keys=[key]) + operation = proto.ConfigRequest.Operation(get=op_get) + return self._client.config(operation).values[0] else: if default is not None: self._checkType(default, "default") - return self._client.config( - "get", keys=[key], optional_values=[cast(Optional[str], default)] - ).optional_values[0] + op_get_with_default = proto.ConfigRequest.GetWithDefault( + pairs=[_optional_key_value(key, cast(Optional[str], default))] + ) + operation = proto.ConfigRequest.Operation(get_with_default=op_get_with_default) + return self._client.config(operation).optional_values[0] get.__doc__ = PySparkRuntimeConfig.get.__doc__ def unset(self, key: str) -> None: - result = self._client.config("unset", keys=[key]) + op_unset = proto.ConfigRequest.Unset(keys=[key]) + operation = proto.ConfigRequest.Operation(unset=op_unset) + result = self._client.config(operation) for warn in result.warnings: warnings.warn(warn) unset.__doc__ = PySparkRuntimeConfig.unset.__doc__ def isModifiable(self, key: str) -> bool: - return self._client.config("is_modifiable", keys=[key]).bools[0] + op_is_modifiable = proto.ConfigRequest.IsModifiable(keys=[key]) + operation = proto.ConfigRequest.Operation(is_modifiable=op_is_modifiable) + return self._client.config(operation).bools[0] isModifiable.__doc__ = PySparkRuntimeConfig.isModifiable.__doc__ @@ -76,6 +88,20 @@ def _checkType(self, obj: Any, identifier: str) -> None: ) +def _optional_value(value: Optional[str]) -> proto.OptionalValue: + return proto.OptionalValue(value=value) + + +def _key_value(key: str, value: Optional[str]) -> proto.KeyValue: + if value is None: + raise IllegalArgumentException(f"requirement failed: value cannot be null for key: {key}") + return proto.KeyValue(key=key, value=value) + + +def _optional_key_value(key: str, value: Optional[str]) -> proto.OptionalKeyValue: + return proto.OptionalKeyValue(key=key, value=_optional_value(value)) + + RuntimeConf.__doc__ = PySparkRuntimeConfig.__doc__ diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 56843821c5ef8..b77c6869279cc 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xfe\x03\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0e\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12\x12\n\x04keys\x18\x04 \x03(\tR\x04keys\x12\x45\n\x0foptional_values\x18\x05 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x0eoptionalValues\x12\x1b\n\x06prefix\x18\x06 \x01(\tH\x00R\x06prefix\x88\x01\x01"\xc7\x01\n\tOperation\x12\x19\n\x15OPERATION_UNSPECIFIED\x10\x00\x12\x11\n\rOPERATION_SET\x10\x01\x12\x11\n\rOPERATION_GET\x10\x02\x12\x18\n\x14OPERATION_GET_OPTION\x10\x03\x12\x15\n\x11OPERATION_GET_ALL\x10\x04\x12\x13\n\x0fOPERATION_UNSET\x10\x05\x12\x16\n\x12OPERATION_CONTAINS\x10\x06\x12\x1b\n\x17OPERATION_IS_MODIFIABLE\x10\x07\x42\t\n\x07_prefix"\xd2\x01\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x12\n\x04keys\x18\x02 \x03(\tR\x04keys\x12\x16\n\x06values\x18\x03 \x03(\tR\x06values\x12\x45\n\x0foptional_values\x18\x04 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x0eoptionalValues\x12\x14\n\x05\x62ools\x18\x05 \x03(\x08R\x05\x62ools\x12\x1a\n\x08warnings\x18\x06 \x03(\tR\x08warnings2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"2\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value"X\n\x10OptionalKeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\xb9\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x1a\xb7\x04\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12\x43\n\x08\x63ontains\x18\x07 \x01(\x0b\x32%.spark.connect.ConfigRequest.ContainsH\x00R\x08\x63ontains\x12P\n\ris_modifiable\x18\x08 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1aG\n\x0eGetWithDefault\x12\x35\n\x05pairs\x18\x01 \x03(\x0b\x32\x1f.spark.connect.OptionalKeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x1e\n\x08\x43ontains\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys"\xd2\x01\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x12\n\x04keys\x18\x02 \x03(\tR\x04keys\x12\x16\n\x06values\x18\x03 \x03(\tR\x06values\x12\x45\n\x0foptional_values\x18\x04 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x0eoptionalValues\x12\x14\n\x05\x62ools\x18\x05 \x03(\x08R\x05\x62ools\x12\x1a\n\x08warnings\x18\x06 \x03(\tR\x08warnings2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -59,10 +59,20 @@ "MetricValue" ] _OPTIONALVALUE = DESCRIPTOR.message_types_by_name["OptionalValue"] +_KEYVALUE = DESCRIPTOR.message_types_by_name["KeyValue"] +_OPTIONALKEYVALUE = DESCRIPTOR.message_types_by_name["OptionalKeyValue"] _CONFIGREQUEST = DESCRIPTOR.message_types_by_name["ConfigRequest"] +_CONFIGREQUEST_OPERATION = _CONFIGREQUEST.nested_types_by_name["Operation"] +_CONFIGREQUEST_SET = _CONFIGREQUEST.nested_types_by_name["Set"] +_CONFIGREQUEST_GET = _CONFIGREQUEST.nested_types_by_name["Get"] +_CONFIGREQUEST_GETWITHDEFAULT = _CONFIGREQUEST.nested_types_by_name["GetWithDefault"] +_CONFIGREQUEST_GETOPTION = _CONFIGREQUEST.nested_types_by_name["GetOption"] +_CONFIGREQUEST_GETALL = _CONFIGREQUEST.nested_types_by_name["GetAll"] +_CONFIGREQUEST_UNSET = _CONFIGREQUEST.nested_types_by_name["Unset"] +_CONFIGREQUEST_CONTAINS = _CONFIGREQUEST.nested_types_by_name["Contains"] +_CONFIGREQUEST_ISMODIFIABLE = _CONFIGREQUEST.nested_types_by_name["IsModifiable"] _CONFIGRESPONSE = DESCRIPTOR.message_types_by_name["ConfigResponse"] _EXPLAIN_EXPLAINMODE = _EXPLAIN.enum_types_by_name["ExplainMode"] -_CONFIGREQUEST_OPERATION = _CONFIGREQUEST.enum_types_by_name["Operation"] Plan = _reflection.GeneratedProtocolMessageType( "Plan", (_message.Message,), @@ -201,16 +211,128 @@ ) _sym_db.RegisterMessage(OptionalValue) +KeyValue = _reflection.GeneratedProtocolMessageType( + "KeyValue", + (_message.Message,), + { + "DESCRIPTOR": _KEYVALUE, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.KeyValue) + }, +) +_sym_db.RegisterMessage(KeyValue) + +OptionalKeyValue = _reflection.GeneratedProtocolMessageType( + "OptionalKeyValue", + (_message.Message,), + { + "DESCRIPTOR": _OPTIONALKEYVALUE, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.OptionalKeyValue) + }, +) +_sym_db.RegisterMessage(OptionalKeyValue) + ConfigRequest = _reflection.GeneratedProtocolMessageType( "ConfigRequest", (_message.Message,), { + "Operation": _reflection.GeneratedProtocolMessageType( + "Operation", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_OPERATION, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.Operation) + }, + ), + "Set": _reflection.GeneratedProtocolMessageType( + "Set", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_SET, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.Set) + }, + ), + "Get": _reflection.GeneratedProtocolMessageType( + "Get", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_GET, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.Get) + }, + ), + "GetWithDefault": _reflection.GeneratedProtocolMessageType( + "GetWithDefault", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_GETWITHDEFAULT, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.GetWithDefault) + }, + ), + "GetOption": _reflection.GeneratedProtocolMessageType( + "GetOption", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_GETOPTION, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.GetOption) + }, + ), + "GetAll": _reflection.GeneratedProtocolMessageType( + "GetAll", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_GETALL, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.GetAll) + }, + ), + "Unset": _reflection.GeneratedProtocolMessageType( + "Unset", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_UNSET, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.Unset) + }, + ), + "Contains": _reflection.GeneratedProtocolMessageType( + "Contains", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_CONTAINS, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.Contains) + }, + ), + "IsModifiable": _reflection.GeneratedProtocolMessageType( + "IsModifiable", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGREQUEST_ISMODIFIABLE, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.IsModifiable) + }, + ), "DESCRIPTOR": _CONFIGREQUEST, "__module__": "spark.connect.base_pb2" # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest) }, ) _sym_db.RegisterMessage(ConfigRequest) +_sym_db.RegisterMessage(ConfigRequest.Operation) +_sym_db.RegisterMessage(ConfigRequest.Set) +_sym_db.RegisterMessage(ConfigRequest.Get) +_sym_db.RegisterMessage(ConfigRequest.GetWithDefault) +_sym_db.RegisterMessage(ConfigRequest.GetOption) +_sym_db.RegisterMessage(ConfigRequest.GetAll) +_sym_db.RegisterMessage(ConfigRequest.Unset) +_sym_db.RegisterMessage(ConfigRequest.Contains) +_sym_db.RegisterMessage(ConfigRequest.IsModifiable) ConfigResponse = _reflection.GeneratedProtocolMessageType( "ConfigResponse", @@ -258,12 +380,32 @@ _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 2107 _OPTIONALVALUE._serialized_start = 2109 _OPTIONALVALUE._serialized_end = 2161 - _CONFIGREQUEST._serialized_start = 2164 - _CONFIGREQUEST._serialized_end = 2674 - _CONFIGREQUEST_OPERATION._serialized_start = 2464 - _CONFIGREQUEST_OPERATION._serialized_end = 2663 - _CONFIGRESPONSE._serialized_start = 2677 - _CONFIGRESPONSE._serialized_end = 2887 - _SPARKCONNECTSERVICE._serialized_start = 2890 - _SPARKCONNECTSERVICE._serialized_end = 3162 + _KEYVALUE._serialized_start = 2163 + _KEYVALUE._serialized_end = 2213 + _OPTIONALKEYVALUE._serialized_start = 2215 + _OPTIONALKEYVALUE._serialized_end = 2303 + _CONFIGREQUEST._serialized_start = 2306 + _CONFIGREQUEST._serialized_end = 3387 + _CONFIGREQUEST_OPERATION._serialized_start = 2486 + _CONFIGREQUEST_OPERATION._serialized_end = 3053 + _CONFIGREQUEST_SET._serialized_start = 3055 + _CONFIGREQUEST_SET._serialized_end = 3107 + _CONFIGREQUEST_GET._serialized_start = 3109 + _CONFIGREQUEST_GET._serialized_end = 3134 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 3136 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3207 + _CONFIGREQUEST_GETOPTION._serialized_start = 3209 + _CONFIGREQUEST_GETOPTION._serialized_end = 3240 + _CONFIGREQUEST_GETALL._serialized_start = 3242 + _CONFIGREQUEST_GETALL._serialized_end = 3290 + _CONFIGREQUEST_UNSET._serialized_start = 3292 + _CONFIGREQUEST_UNSET._serialized_end = 3319 + _CONFIGREQUEST_CONTAINS._serialized_start = 3321 + _CONFIGREQUEST_CONTAINS._serialized_end = 3351 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3353 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3387 + _CONFIGRESPONSE._serialized_start = 3390 + _CONFIGRESPONSE._serialized_end = 3600 + _SPARKCONNECTSERVICE._serialized_start = 3603 + _SPARKCONNECTSERVICE._serialized_end = 3875 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index a72d2e79bec22..5288ce07c2b94 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -595,47 +595,293 @@ class OptionalValue(google.protobuf.message.Message): global___OptionalValue = OptionalValue +class KeyValue(google.protobuf.message.Message): + """The key-value pair for the config request and response.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + value: builtins.str + def __init__( + self, + *, + key: builtins.str = ..., + value: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + +global___KeyValue = KeyValue + +class OptionalKeyValue(google.protobuf.message.Message): + """The key-value pair for the config request and response when the value can be optional.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + @property + def value(self) -> global___OptionalValue: ... + def __init__( + self, + *, + key: builtins.str = ..., + value: global___OptionalValue | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["value", b"value"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + +global___OptionalKeyValue = OptionalKeyValue + class ConfigRequest(google.protobuf.message.Message): """Request to update or fetch the configurations.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - class _Operation: - ValueType = typing.NewType("ValueType", builtins.int) - V: typing_extensions.TypeAlias = ValueType + class Operation(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor - class _OperationEnumTypeWrapper( - google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ - ConfigRequest._Operation.ValueType - ], - builtins.type, - ): # noqa: F821 - DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor - OPERATION_UNSPECIFIED: ConfigRequest._Operation.ValueType # 0 - OPERATION_SET: ConfigRequest._Operation.ValueType # 1 - OPERATION_GET: ConfigRequest._Operation.ValueType # 2 - OPERATION_GET_OPTION: ConfigRequest._Operation.ValueType # 3 - OPERATION_GET_ALL: ConfigRequest._Operation.ValueType # 4 - OPERATION_UNSET: ConfigRequest._Operation.ValueType # 5 - OPERATION_CONTAINS: ConfigRequest._Operation.ValueType # 6 - OPERATION_IS_MODIFIABLE: ConfigRequest._Operation.ValueType # 7 - - class Operation(_Operation, metaclass=_OperationEnumTypeWrapper): ... - OPERATION_UNSPECIFIED: ConfigRequest.Operation.ValueType # 0 - OPERATION_SET: ConfigRequest.Operation.ValueType # 1 - OPERATION_GET: ConfigRequest.Operation.ValueType # 2 - OPERATION_GET_OPTION: ConfigRequest.Operation.ValueType # 3 - OPERATION_GET_ALL: ConfigRequest.Operation.ValueType # 4 - OPERATION_UNSET: ConfigRequest.Operation.ValueType # 5 - OPERATION_CONTAINS: ConfigRequest.Operation.ValueType # 6 - OPERATION_IS_MODIFIABLE: ConfigRequest.Operation.ValueType # 7 + SET_FIELD_NUMBER: builtins.int + GET_FIELD_NUMBER: builtins.int + GET_WITH_DEFAULT_FIELD_NUMBER: builtins.int + GET_OPTION_FIELD_NUMBER: builtins.int + GET_ALL_FIELD_NUMBER: builtins.int + UNSET_FIELD_NUMBER: builtins.int + CONTAINS_FIELD_NUMBER: builtins.int + IS_MODIFIABLE_FIELD_NUMBER: builtins.int + @property + def set(self) -> global___ConfigRequest.Set: ... + @property + def get(self) -> global___ConfigRequest.Get: ... + @property + def get_with_default(self) -> global___ConfigRequest.GetWithDefault: ... + @property + def get_option(self) -> global___ConfigRequest.GetOption: ... + @property + def get_all(self) -> global___ConfigRequest.GetAll: ... + @property + def unset(self) -> global___ConfigRequest.Unset: ... + @property + def contains(self) -> global___ConfigRequest.Contains: ... + @property + def is_modifiable(self) -> global___ConfigRequest.IsModifiable: ... + def __init__( + self, + *, + set: global___ConfigRequest.Set | None = ..., + get: global___ConfigRequest.Get | None = ..., + get_with_default: global___ConfigRequest.GetWithDefault | None = ..., + get_option: global___ConfigRequest.GetOption | None = ..., + get_all: global___ConfigRequest.GetAll | None = ..., + unset: global___ConfigRequest.Unset | None = ..., + contains: global___ConfigRequest.Contains | None = ..., + is_modifiable: global___ConfigRequest.IsModifiable | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "contains", + b"contains", + "get", + b"get", + "get_all", + b"get_all", + "get_option", + b"get_option", + "get_with_default", + b"get_with_default", + "is_modifiable", + b"is_modifiable", + "op_type", + b"op_type", + "set", + b"set", + "unset", + b"unset", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "contains", + b"contains", + "get", + b"get", + "get_all", + b"get_all", + "get_option", + b"get_option", + "get_with_default", + b"get_with_default", + "is_modifiable", + b"is_modifiable", + "op_type", + b"op_type", + "set", + b"set", + "unset", + b"unset", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["op_type", b"op_type"] + ) -> typing_extensions.Literal[ + "set", + "get", + "get_with_default", + "get_option", + "get_all", + "unset", + "contains", + "is_modifiable", + ] | None: ... + + class Set(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PAIRS_FIELD_NUMBER: builtins.int + @property + def pairs( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[global___KeyValue]: + """(Required) The config key-value pairs to set.""" + def __init__( + self, + *, + pairs: collections.abc.Iterable[global___KeyValue] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... + + class Get(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEYS_FIELD_NUMBER: builtins.int + @property + def keys( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Required) The config keys to get.""" + def __init__( + self, + *, + keys: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + + class GetWithDefault(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PAIRS_FIELD_NUMBER: builtins.int + @property + def pairs( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___OptionalKeyValue + ]: + """(Required) The config key-value paris to get. The value will be used as the default value.""" + def __init__( + self, + *, + pairs: collections.abc.Iterable[global___OptionalKeyValue] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... + + class GetOption(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEYS_FIELD_NUMBER: builtins.int + @property + def keys( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Required) The config keys to get optionally.""" + def __init__( + self, + *, + keys: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + + class GetAll(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PREFIX_FIELD_NUMBER: builtins.int + prefix: builtins.str + """(Optional) The prefix of the config key to get.""" + def __init__( + self, + *, + prefix: builtins.str | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["_prefix", b"_prefix", "prefix", b"prefix"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["_prefix", b"_prefix", "prefix", b"prefix"] + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_prefix", b"_prefix"] + ) -> typing_extensions.Literal["prefix"] | None: ... + + class Unset(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEYS_FIELD_NUMBER: builtins.int + @property + def keys( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Required) The config keys to unset.""" + def __init__( + self, + *, + keys: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + + class Contains(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEYS_FIELD_NUMBER: builtins.int + @property + def keys( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Required) The config keys to check if the config contains them.""" + def __init__( + self, + *, + keys: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + + class IsModifiable(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEYS_FIELD_NUMBER: builtins.int + @property + def keys( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Required) The config keys to check the config is modifiable.""" + def __init__( + self, + *, + keys: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... CLIENT_ID_FIELD_NUMBER: builtins.int USER_CONTEXT_FIELD_NUMBER: builtins.int OPERATION_FIELD_NUMBER: builtins.int - KEYS_FIELD_NUMBER: builtins.int - OPTIONAL_VALUES_FIELD_NUMBER: builtins.int - PREFIX_FIELD_NUMBER: builtins.int client_id: builtins.str """(Required) @@ -645,74 +891,28 @@ class ConfigRequest(google.protobuf.message.Message): @property def user_context(self) -> global___UserContext: """(Required) User context""" - operation: global___ConfigRequest.Operation.ValueType @property - def keys( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: - """(Optional) - - Identify which keys will be updated or fetched. - Required when the 'operation' is 'SET', 'GET', 'GET_OPTION', 'UNSET', - 'CONTAINS', 'IS_MODIFIABLE'. - """ - @property - def optional_values( - self, - ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ - global___OptionalValue - ]: - """(Optional) - - Corresponding values to the keys. - Required when the 'operation' is 'SET'. - Optional when the 'operation' is 'GET', the values here will be used - as the default values. - """ - prefix: builtins.str - """(Optional) - - Only used when the 'operation' is 'GET_ALL'. - If prefix is given, only parameters that start with the prefix will be returned. - """ + def operation(self) -> global___ConfigRequest.Operation: + """(Required) The operation for the config.""" def __init__( self, *, client_id: builtins.str = ..., user_context: global___UserContext | None = ..., - operation: global___ConfigRequest.Operation.ValueType = ..., - keys: collections.abc.Iterable[builtins.str] | None = ..., - optional_values: collections.abc.Iterable[global___OptionalValue] | None = ..., - prefix: builtins.str | None = ..., + operation: global___ConfigRequest.Operation | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ - "_prefix", b"_prefix", "prefix", b"prefix", "user_context", b"user_context" + "operation", b"operation", "user_context", b"user_context" ], ) -> builtins.bool: ... def ClearField( self, field_name: typing_extensions.Literal[ - "_prefix", - b"_prefix", - "client_id", - b"client_id", - "keys", - b"keys", - "operation", - b"operation", - "optional_values", - b"optional_values", - "prefix", - b"prefix", - "user_context", - b"user_context", + "client_id", b"client_id", "operation", b"operation", "user_context", b"user_context" ], ) -> None: ... - def WhichOneof( - self, oneof_group: typing_extensions.Literal["_prefix", b"_prefix"] - ) -> typing_extensions.Literal["prefix"] | None: ... global___ConfigRequest = ConfigRequest From f7b6ac0e8d535ac9d4ca4e95fbc7bf3bc976a1ab Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Feb 2023 14:54:19 -0800 Subject: [PATCH 06/14] Fix. --- .../main/protobuf/spark/connect/base.proto | 64 ++-- .../service/SparkConnectConfigHandler.scala | 122 ++++---- python/pyspark/sql/connect/client.py | 99 +++++-- python/pyspark/sql/connect/conf.py | 16 +- python/pyspark/sql/connect/proto/base_pb2.py | 125 +++++++- python/pyspark/sql/connect/proto/base_pb2.pyi | 277 ++++++++++++++---- 6 files changed, 548 insertions(+), 155 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 8bf705460782c..30e2baff3832b 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -272,32 +272,56 @@ message ConfigRequest { message ConfigResponse { string client_id = 1; - // (Optional) - // - // Available when the operation is 'GET_ALL'. - repeated string keys = 2; + // (Required) The operation for the config. + Operation operation = 2; // (Optional) // - // Available when the operation is 'GET_ALL'. - // Optional when the 'operation' is 'GET' without the default values. - repeated string values = 3; + // Warning messages for deprecated or unsupported configurations. + repeated string warnings = 3; - // (Optional) - // - // Available when the operation is 'GET_OPTION'. - // Optional when the operation is 'GET' with the default values. - repeated OptionalValue optional_values = 4; + message Operation { + oneof op_type { + Set set = 1; + Get get = 2; + GetWithDefault get_with_default = 3; + GetOption get_option = 4; + GetAll get_all = 5; + Unset unset = 6; + Contains contains = 7; + IsModifiable is_modifiable = 8; + } + } - // (Optional) - // - // Available when the operation is 'CONTAINS', 'IS_MODIFIABLE'. - repeated bool bools = 5; + message Set { + } - // (Optional) - // - // Warning messages for deprecated or unsupported configurations. - repeated string warnings = 6; + message Get { + repeated string values = 1; + } + + message GetWithDefault { + repeated OptionalValue values = 1; + } + + message GetOption { + repeated OptionalValue values = 1; + } + + message GetAll { + repeated KeyValue pairs = 1; + } + + message Unset { + } + + message Contains { + repeated bool bools = 1; + } + + message IsModifiable { + repeated bool bools = 1; + } } // Main interface for the SparkConnect service. diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index d81e6be9789a8..d3fc1959407ea 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -35,7 +35,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes .getOrCreateIsolatedSession(request.getUserContext.getUserId, request.getClientId) .session - val builder = request.getOperation.getOpTypeCase match { + val (operation, warnings) = request.getOperation.getOpTypeCase match { case proto.ConfigRequest.Operation.OpTypeCase.SET => handleSet(request.getOperation.getSet, session.conf) case proto.ConfigRequest.Operation.OpTypeCase.GET => @@ -55,64 +55,77 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes case _ => throw new UnsupportedOperationException(s"${request.getOperation} not supported.") } - builder.setClientId(request.getClientId) - responseObserver.onNext(builder.build()) + val response = proto.ConfigResponse + .newBuilder() + .setClientId(request.getClientId) + .setOperation(operation) + .addAllWarnings(warnings.asJava) + .build() + responseObserver.onNext(response) responseObserver.onCompleted() } private def handleSet( operation: proto.ConfigRequest.Set, - conf: RuntimeConfig): proto.ConfigResponse.Builder = { - val builder = proto.ConfigResponse.newBuilder() - operation.getPairsList.asScala.iterator.foreach { pair => + conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { + val builder = proto.ConfigResponse.Operation.newBuilder() + val response = proto.ConfigResponse.Set.newBuilder() + val warnings = operation.getPairsList.asScala.flatMap { pair => val key = pair.getKey val value = pair.getValue conf.set(key, value) - getWarning(key).foreach(builder.addWarnings) + getWarning(key) } - builder + builder.setSet(response.build()) + (builder.build(), warnings) } private def handleGet( operation: proto.ConfigRequest.Get, - conf: RuntimeConfig): proto.ConfigResponse.Builder = { - val builder = proto.ConfigResponse.newBuilder() - operation.getKeysList.asScala.iterator.foreach { key => - builder.addValues(conf.get(key)) - getWarning(key).foreach(builder.addWarnings) + conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { + val builder = proto.ConfigResponse.Operation.newBuilder() + val response = proto.ConfigResponse.Get.newBuilder() + val warnings = operation.getKeysList.asScala.flatMap { key => + response.addValues(conf.get(key)) + getWarning(key) } - builder + builder.setGet(response.build()) + (builder.build(), warnings) } private def handleGetWithDefault( operation: proto.ConfigRequest.GetWithDefault, - conf: RuntimeConfig): proto.ConfigResponse.Builder = { - val builder = proto.ConfigResponse.newBuilder() - operation.getPairsList.asScala.iterator.foreach { pair => + conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { + val builder = proto.ConfigResponse.Operation.newBuilder() + val response = proto.ConfigResponse.GetWithDefault.newBuilder() + val warnings = operation.getPairsList.asScala.flatMap { pair => val key = pair.getKey val value = SparkConnectConfigHandler.toOption(pair.getValue).orNull - builder.addOptionalValues( - SparkConnectConfigHandler.toOptionalValue(Option(conf.get(key, value)))) - getWarning(key).foreach(builder.addWarnings) + response.addValues(SparkConnectConfigHandler.toOptionalValue(Option(conf.get(key, value)))) + getWarning(key) } - builder + builder.setGetWithDefault(response.build()) + (builder.build(), warnings) } private def handleGetOption( operation: proto.ConfigRequest.GetOption, - conf: RuntimeConfig): proto.ConfigResponse.Builder = { - val builder = proto.ConfigResponse.newBuilder() - operation.getKeysList.asScala.iterator.foreach { key => - builder.addOptionalValues(SparkConnectConfigHandler.toOptionalValue(conf.getOption(key))) - getWarning(key).foreach(builder.addWarnings) + conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { + val builder = proto.ConfigResponse.Operation.newBuilder() + val response = proto.ConfigResponse.GetOption.newBuilder() + val warnings = operation.getKeysList.asScala.flatMap { key => + response.addValues(SparkConnectConfigHandler.toOptionalValue(conf.getOption(key))) + getWarning(key) } - builder + builder.setGetOption(response.build()) + (builder.build(), warnings) } private def handleGetAll( operation: proto.ConfigRequest.GetAll, - conf: RuntimeConfig): proto.ConfigResponse.Builder = { - val builder = proto.ConfigResponse.newBuilder() + conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { + val builder = proto.ConfigResponse.Operation.newBuilder() + val response = proto.ConfigResponse.GetAll.newBuilder() val results = if (operation.hasPrefix) { val prefix = operation.getPrefix conf.getAll.iterator @@ -121,44 +134,51 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes } else { conf.getAll.iterator } - results.foreach { case (key, value) => - builder.addKeys(key).addValues(value) - getWarning(key).foreach(builder.addWarnings) - } - builder + val warnings = results.flatMap { case (key, value) => + response.addPairs(proto.KeyValue.newBuilder().setKey(key).setValue(value).build()) + getWarning(key) + }.toSeq + builder.setGetAll(response.build()) + (builder.build(), warnings) } private def handleUnset( operation: proto.ConfigRequest.Unset, - conf: RuntimeConfig): proto.ConfigResponse.Builder = { - val builder = proto.ConfigResponse.newBuilder() - operation.getKeysList.asScala.iterator.foreach { key => + conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { + val builder = proto.ConfigResponse.Operation.newBuilder() + val response = proto.ConfigResponse.Unset.newBuilder() + val warnings = operation.getKeysList.asScala.flatMap { key => conf.unset(key) - getWarning(key).foreach(builder.addWarnings) + getWarning(key) } - builder + builder.setUnset(response.build()) + (builder.build(), warnings) } private def handleContains( operation: proto.ConfigRequest.Contains, - conf: RuntimeConfig): proto.ConfigResponse.Builder = { - val builder = proto.ConfigResponse.newBuilder() - operation.getKeysList.asScala.iterator.foreach { key => - builder.addBools(conf.contains(key)) - getWarning(key).foreach(builder.addWarnings) + conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { + val builder = proto.ConfigResponse.Operation.newBuilder() + val response = proto.ConfigResponse.Contains.newBuilder() + val warnings = operation.getKeysList.asScala.flatMap { key => + response.addBools(conf.contains(key)) + getWarning(key) } - builder + builder.setContains(response.build()) + (builder.build(), warnings) } private def handleIsModifiable( operation: proto.ConfigRequest.IsModifiable, - conf: RuntimeConfig): proto.ConfigResponse.Builder = { - val builder = proto.ConfigResponse.newBuilder() - operation.getKeysList.asScala.iterator.foreach { key => - builder.addBools(conf.isModifiable(key)) - getWarning(key).foreach(builder.addWarnings) + conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { + val builder = proto.ConfigResponse.Operation.newBuilder() + val response = proto.ConfigResponse.IsModifiable.newBuilder() + val warnings = operation.getKeysList.asScala.flatMap { key => + response.addBools(conf.isModifiable(key)) + getWarning(key) } - builder + builder.setIsModifiable(response.build()) + (builder.build(), warnings) } private def getWarning(key: String): Option[String] = { diff --git a/python/pyspark/sql/connect/client.py b/python/pyspark/sql/connect/client.py index 1eaf46541bf45..be8c7471cd4bd 100644 --- a/python/pyspark/sql/connect/client.py +++ b/python/pyspark/sql/connect/client.py @@ -403,31 +403,88 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": class ConfigResult: - def __init__( - self, - keys: List[str], - values: List[str], - optional_values: List[Optional[str]], - bools: List[bool], - warnings: List[str], - ): - self.keys = keys - self.values = values - self.optional_values = optional_values - self.bools = bools + class Operation: + pass + + class Set(Operation): + pass + + class Get(Operation): + def __init__(self, values: List[str]): + self.values = values + + class GetWithDefault(Operation): + def __init__(self, values: List[Optional[str]]): + self.values = values + + class GetOption(Operation): + def __init__(self, values: List[Optional[str]]): + self.values = values + + class GetAll(Operation): + def __init__(self, pairs: List[Tuple[str, str]]): + self.pairs = pairs + + class Unset(Operation): + pass + + class Contains(Operation): + def __init__(self, bools: List[bool]): + self.bools = bools + + class IsModifiable(Operation): + def __init__(self, bools: List[bool]): + self.bools = bools + + def __init__(self, operation: Operation, warnings: List[str]): + self.operation = operation self.warnings = warnings @classmethod def fromProto(cls, pb: pb2.ConfigResponse) -> "ConfigResult": - return ConfigResult( - keys=list(pb.keys), - values=list(pb.values), - optional_values=[ - value.value if value.HasField("value") else None for value in pb.optional_values - ], - bools=list(pb.bools), - warnings=list(pb.warnings), - ) + warnings = list(pb.warnings) + op_type = pb.operation.WhichOneof("op_type") + if op_type == "set": + return ConfigResult(ConfigResult.Set(), warnings) + elif op_type == "get": + return ConfigResult(ConfigResult.Get(list(pb.operation.get.values)), warnings) + elif op_type == "get_with_default": + return ConfigResult( + ConfigResult.GetWithDefault( + [ + value.value if value.HasField("value") else None + for value in pb.operation.get_with_default.values + ] + ), + warnings, + ) + elif op_type == "get_option": + return ConfigResult( + ConfigResult.GetOption( + [ + value.value if value.HasField("value") else None + for value in pb.operation.get_with_default.values + ] + ), + warnings, + ) + elif op_type == "get_all": + return ConfigResult( + ConfigResult.GetAll( + [(pair.key, pair.value) for pair in pb.operation.get_all.pairs] + ), + warnings, + ) + elif op_type == "unset": + return ConfigResult(ConfigResult.Unset(), warnings) + elif op_type == "contains": + return ConfigResult(ConfigResult.Contains(list(pb.operation.contains.bools)), warnings) + elif op_type == "is_modifiable": + return ConfigResult( + ConfigResult.IsModifiable(list(pb.operation.is_modifiable.bools)), warnings + ) + else: + raise ValueError(f"Unknown op_type in ConfigResponse: {op_type}") class SparkConnectClient(object): diff --git a/python/pyspark/sql/connect/conf.py b/python/pyspark/sql/connect/conf.py index adcb7197d48d8..2a4ab3fdd4746 100644 --- a/python/pyspark/sql/connect/conf.py +++ b/python/pyspark/sql/connect/conf.py @@ -22,7 +22,7 @@ from pyspark.errors import IllegalArgumentException from pyspark.sql.conf import RuntimeConfig as PySparkRuntimeConfig from pyspark.sql.connect import proto -from pyspark.sql.connect.client import SparkConnectClient +from pyspark.sql.connect.client import ConfigResult, SparkConnectClient class RuntimeConf: @@ -40,6 +40,7 @@ def set(self, key: str, value: Union[str, int, bool]) -> None: op_set = proto.ConfigRequest.Set(pairs=[_key_value(key, value)]) operation = proto.ConfigRequest.Operation(set=op_set) result = self._client.config(operation) + assert isinstance(result.operation, ConfigResult.Set) for warn in result.warnings: warnings.warn(warn) @@ -52,7 +53,9 @@ def get( if default is _NoValue: op_get = proto.ConfigRequest.Get(keys=[key]) operation = proto.ConfigRequest.Operation(get=op_get) - return self._client.config(operation).values[0] + result = self._client.config(operation) + assert isinstance(result.operation, ConfigResult.Get) + return result.operation.values[0] else: if default is not None: self._checkType(default, "default") @@ -60,7 +63,9 @@ def get( pairs=[_optional_key_value(key, cast(Optional[str], default))] ) operation = proto.ConfigRequest.Operation(get_with_default=op_get_with_default) - return self._client.config(operation).optional_values[0] + result = self._client.config(operation) + assert isinstance(result.operation, ConfigResult.GetWithDefault) + return result.operation.values[0] get.__doc__ = PySparkRuntimeConfig.get.__doc__ @@ -68,6 +73,7 @@ def unset(self, key: str) -> None: op_unset = proto.ConfigRequest.Unset(keys=[key]) operation = proto.ConfigRequest.Operation(unset=op_unset) result = self._client.config(operation) + assert isinstance(result.operation, ConfigResult.Unset) for warn in result.warnings: warnings.warn(warn) @@ -76,7 +82,9 @@ def unset(self, key: str) -> None: def isModifiable(self, key: str) -> bool: op_is_modifiable = proto.ConfigRequest.IsModifiable(keys=[key]) operation = proto.ConfigRequest.Operation(is_modifiable=op_is_modifiable) - return self._client.config(operation).bools[0] + result = self._client.config(operation) + assert isinstance(result.operation, ConfigResult.IsModifiable) + return result.operation.bools[0] isModifiable.__doc__ = PySparkRuntimeConfig.isModifiable.__doc__ diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index b77c6869279cc..17797cc122f46 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"2\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value"X\n\x10OptionalKeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\xb9\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x1a\xb7\x04\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12\x43\n\x08\x63ontains\x18\x07 \x01(\x0b\x32%.spark.connect.ConfigRequest.ContainsH\x00R\x08\x63ontains\x12P\n\ris_modifiable\x18\x08 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1aG\n\x0eGetWithDefault\x12\x35\n\x05pairs\x18\x01 \x03(\x0b\x32\x1f.spark.connect.OptionalKeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x1e\n\x08\x43ontains\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys"\xd2\x01\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x12\n\x04keys\x18\x02 \x03(\tR\x04keys\x12\x16\n\x06values\x18\x03 \x03(\tR\x06values\x12\x45\n\x0foptional_values\x18\x04 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x0eoptionalValues\x12\x14\n\x05\x62ools\x18\x05 \x03(\x08R\x05\x62ools\x12\x1a\n\x08warnings\x18\x06 \x03(\tR\x08warnings2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"2\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value"X\n\x10OptionalKeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\xb9\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x1a\xb7\x04\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12\x43\n\x08\x63ontains\x18\x07 \x01(\x0b\x32%.spark.connect.ConfigRequest.ContainsH\x00R\x08\x63ontains\x12P\n\ris_modifiable\x18\x08 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1aG\n\x0eGetWithDefault\x12\x35\n\x05pairs\x18\x01 \x03(\x0b\x32\x1f.spark.connect.OptionalKeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x1e\n\x08\x43ontains\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys"\x8d\x08\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\'.spark.connect.ConfigResponse.OperationR\toperation\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings\x1a\xbf\x04\n\tOperation\x12\x35\n\x03set\x18\x01 \x01(\x0b\x32!.spark.connect.ConfigResponse.SetH\x00R\x03set\x12\x35\n\x03get\x18\x02 \x01(\x0b\x32!.spark.connect.ConfigResponse.GetH\x00R\x03get\x12X\n\x10get_with_default\x18\x03 \x01(\x0b\x32,.spark.connect.ConfigResponse.GetWithDefaultH\x00R\x0egetWithDefault\x12H\n\nget_option\x18\x04 \x01(\x0b\x32\'.spark.connect.ConfigResponse.GetOptionH\x00R\tgetOption\x12?\n\x07get_all\x18\x05 \x01(\x0b\x32$.spark.connect.ConfigResponse.GetAllH\x00R\x06getAll\x12;\n\x05unset\x18\x06 \x01(\x0b\x32#.spark.connect.ConfigResponse.UnsetH\x00R\x05unset\x12\x44\n\x08\x63ontains\x18\x07 \x01(\x0b\x32&.spark.connect.ConfigResponse.ContainsH\x00R\x08\x63ontains\x12Q\n\ris_modifiable\x18\x08 \x01(\x0b\x32*.spark.connect.ConfigResponse.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x05\n\x03Set\x1a\x1d\n\x03Get\x12\x16\n\x06values\x18\x01 \x03(\tR\x06values\x1a\x46\n\x0eGetWithDefault\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x41\n\tGetOption\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x37\n\x06GetAll\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x07\n\x05Unset\x1a \n\x08\x43ontains\x12\x14\n\x05\x62ools\x18\x01 \x03(\x08R\x05\x62ools\x1a$\n\x0cIsModifiable\x12\x14\n\x05\x62ools\x18\x01 \x03(\x08R\x05\x62ools2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -72,6 +72,15 @@ _CONFIGREQUEST_CONTAINS = _CONFIGREQUEST.nested_types_by_name["Contains"] _CONFIGREQUEST_ISMODIFIABLE = _CONFIGREQUEST.nested_types_by_name["IsModifiable"] _CONFIGRESPONSE = DESCRIPTOR.message_types_by_name["ConfigResponse"] +_CONFIGRESPONSE_OPERATION = _CONFIGRESPONSE.nested_types_by_name["Operation"] +_CONFIGRESPONSE_SET = _CONFIGRESPONSE.nested_types_by_name["Set"] +_CONFIGRESPONSE_GET = _CONFIGRESPONSE.nested_types_by_name["Get"] +_CONFIGRESPONSE_GETWITHDEFAULT = _CONFIGRESPONSE.nested_types_by_name["GetWithDefault"] +_CONFIGRESPONSE_GETOPTION = _CONFIGRESPONSE.nested_types_by_name["GetOption"] +_CONFIGRESPONSE_GETALL = _CONFIGRESPONSE.nested_types_by_name["GetAll"] +_CONFIGRESPONSE_UNSET = _CONFIGRESPONSE.nested_types_by_name["Unset"] +_CONFIGRESPONSE_CONTAINS = _CONFIGRESPONSE.nested_types_by_name["Contains"] +_CONFIGRESPONSE_ISMODIFIABLE = _CONFIGRESPONSE.nested_types_by_name["IsModifiable"] _EXPLAIN_EXPLAINMODE = _EXPLAIN.enum_types_by_name["ExplainMode"] Plan = _reflection.GeneratedProtocolMessageType( "Plan", @@ -338,12 +347,102 @@ "ConfigResponse", (_message.Message,), { + "Operation": _reflection.GeneratedProtocolMessageType( + "Operation", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_OPERATION, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Operation) + }, + ), + "Set": _reflection.GeneratedProtocolMessageType( + "Set", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_SET, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Set) + }, + ), + "Get": _reflection.GeneratedProtocolMessageType( + "Get", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_GET, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Get) + }, + ), + "GetWithDefault": _reflection.GeneratedProtocolMessageType( + "GetWithDefault", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_GETWITHDEFAULT, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.GetWithDefault) + }, + ), + "GetOption": _reflection.GeneratedProtocolMessageType( + "GetOption", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_GETOPTION, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.GetOption) + }, + ), + "GetAll": _reflection.GeneratedProtocolMessageType( + "GetAll", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_GETALL, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.GetAll) + }, + ), + "Unset": _reflection.GeneratedProtocolMessageType( + "Unset", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_UNSET, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Unset) + }, + ), + "Contains": _reflection.GeneratedProtocolMessageType( + "Contains", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_CONTAINS, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Contains) + }, + ), + "IsModifiable": _reflection.GeneratedProtocolMessageType( + "IsModifiable", + (_message.Message,), + { + "DESCRIPTOR": _CONFIGRESPONSE_ISMODIFIABLE, + "__module__": "spark.connect.base_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.IsModifiable) + }, + ), "DESCRIPTOR": _CONFIGRESPONSE, "__module__": "spark.connect.base_pb2" # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse) }, ) _sym_db.RegisterMessage(ConfigResponse) +_sym_db.RegisterMessage(ConfigResponse.Operation) +_sym_db.RegisterMessage(ConfigResponse.Set) +_sym_db.RegisterMessage(ConfigResponse.Get) +_sym_db.RegisterMessage(ConfigResponse.GetWithDefault) +_sym_db.RegisterMessage(ConfigResponse.GetOption) +_sym_db.RegisterMessage(ConfigResponse.GetAll) +_sym_db.RegisterMessage(ConfigResponse.Unset) +_sym_db.RegisterMessage(ConfigResponse.Contains) +_sym_db.RegisterMessage(ConfigResponse.IsModifiable) _SPARKCONNECTSERVICE = DESCRIPTOR.services_by_name["SparkConnectService"] if _descriptor._USE_C_DESCRIPTORS == False: @@ -405,7 +504,25 @@ _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3353 _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3387 _CONFIGRESPONSE._serialized_start = 3390 - _CONFIGRESPONSE._serialized_end = 3600 - _SPARKCONNECTSERVICE._serialized_start = 3603 - _SPARKCONNECTSERVICE._serialized_end = 3875 + _CONFIGRESPONSE._serialized_end = 4427 + _CONFIGRESPONSE_OPERATION._serialized_start = 3537 + _CONFIGRESPONSE_OPERATION._serialized_end = 4112 + _CONFIGRESPONSE_SET._serialized_start = 3055 + _CONFIGRESPONSE_SET._serialized_end = 3060 + _CONFIGRESPONSE_GET._serialized_start = 4121 + _CONFIGRESPONSE_GET._serialized_end = 4150 + _CONFIGRESPONSE_GETWITHDEFAULT._serialized_start = 4152 + _CONFIGRESPONSE_GETWITHDEFAULT._serialized_end = 4222 + _CONFIGRESPONSE_GETOPTION._serialized_start = 4224 + _CONFIGRESPONSE_GETOPTION._serialized_end = 4289 + _CONFIGRESPONSE_GETALL._serialized_start = 4291 + _CONFIGRESPONSE_GETALL._serialized_end = 4346 + _CONFIGRESPONSE_UNSET._serialized_start = 3292 + _CONFIGRESPONSE_UNSET._serialized_end = 3299 + _CONFIGRESPONSE_CONTAINS._serialized_start = 4357 + _CONFIGRESPONSE_CONTAINS._serialized_end = 4389 + _CONFIGRESPONSE_ISMODIFIABLE._serialized_start = 4391 + _CONFIGRESPONSE_ISMODIFIABLE._serialized_end = 4427 + _SPARKCONNECTSERVICE._serialized_start = 4430 + _SPARKCONNECTSERVICE._serialized_end = 4702 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 5288ce07c2b94..40918e0b66de1 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -921,49 +921,227 @@ class ConfigResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - CLIENT_ID_FIELD_NUMBER: builtins.int - KEYS_FIELD_NUMBER: builtins.int - VALUES_FIELD_NUMBER: builtins.int - OPTIONAL_VALUES_FIELD_NUMBER: builtins.int - BOOLS_FIELD_NUMBER: builtins.int - WARNINGS_FIELD_NUMBER: builtins.int - client_id: builtins.str - @property - def keys( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: - """(Optional) + class Operation(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor - Available when the operation is 'GET_ALL'. - """ - @property - def values( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: - """(Optional) + SET_FIELD_NUMBER: builtins.int + GET_FIELD_NUMBER: builtins.int + GET_WITH_DEFAULT_FIELD_NUMBER: builtins.int + GET_OPTION_FIELD_NUMBER: builtins.int + GET_ALL_FIELD_NUMBER: builtins.int + UNSET_FIELD_NUMBER: builtins.int + CONTAINS_FIELD_NUMBER: builtins.int + IS_MODIFIABLE_FIELD_NUMBER: builtins.int + @property + def set(self) -> global___ConfigResponse.Set: ... + @property + def get(self) -> global___ConfigResponse.Get: ... + @property + def get_with_default(self) -> global___ConfigResponse.GetWithDefault: ... + @property + def get_option(self) -> global___ConfigResponse.GetOption: ... + @property + def get_all(self) -> global___ConfigResponse.GetAll: ... + @property + def unset(self) -> global___ConfigResponse.Unset: ... + @property + def contains(self) -> global___ConfigResponse.Contains: ... + @property + def is_modifiable(self) -> global___ConfigResponse.IsModifiable: ... + def __init__( + self, + *, + set: global___ConfigResponse.Set | None = ..., + get: global___ConfigResponse.Get | None = ..., + get_with_default: global___ConfigResponse.GetWithDefault | None = ..., + get_option: global___ConfigResponse.GetOption | None = ..., + get_all: global___ConfigResponse.GetAll | None = ..., + unset: global___ConfigResponse.Unset | None = ..., + contains: global___ConfigResponse.Contains | None = ..., + is_modifiable: global___ConfigResponse.IsModifiable | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "contains", + b"contains", + "get", + b"get", + "get_all", + b"get_all", + "get_option", + b"get_option", + "get_with_default", + b"get_with_default", + "is_modifiable", + b"is_modifiable", + "op_type", + b"op_type", + "set", + b"set", + "unset", + b"unset", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "contains", + b"contains", + "get", + b"get", + "get_all", + b"get_all", + "get_option", + b"get_option", + "get_with_default", + b"get_with_default", + "is_modifiable", + b"is_modifiable", + "op_type", + b"op_type", + "set", + b"set", + "unset", + b"unset", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["op_type", b"op_type"] + ) -> typing_extensions.Literal[ + "set", + "get", + "get_with_default", + "get_option", + "get_all", + "unset", + "contains", + "is_modifiable", + ] | None: ... - Available when the operation is 'GET_ALL'. - Optional when the 'operation' is 'GET' without the default values. - """ - @property - def optional_values( - self, - ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ - global___OptionalValue - ]: - """(Optional) + class Set(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor - Available when the operation is 'GET_OPTION'. - Optional when the operation is 'GET' with the default values. - """ - @property - def bools( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.bool]: - """(Optional) + def __init__( + self, + ) -> None: ... - Available when the operation is 'CONTAINS', 'IS_MODIFIABLE'. - """ + class Get(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["values", b"values"] + ) -> None: ... + + class GetWithDefault(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___OptionalValue + ]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[global___OptionalValue] | None = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["values", b"values"] + ) -> None: ... + + class GetOption(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___OptionalValue + ]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[global___OptionalValue] | None = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["values", b"values"] + ) -> None: ... + + class GetAll(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PAIRS_FIELD_NUMBER: builtins.int + @property + def pairs( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___KeyValue + ]: ... + def __init__( + self, + *, + pairs: collections.abc.Iterable[global___KeyValue] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... + + class Unset(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + def __init__( + self, + ) -> None: ... + + class Contains(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + BOOLS_FIELD_NUMBER: builtins.int + @property + def bools( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.bool]: ... + def __init__( + self, + *, + bools: collections.abc.Iterable[builtins.bool] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["bools", b"bools"]) -> None: ... + + class IsModifiable(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + BOOLS_FIELD_NUMBER: builtins.int + @property + def bools( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.bool]: ... + def __init__( + self, + *, + bools: collections.abc.Iterable[builtins.bool] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["bools", b"bools"]) -> None: ... + + CLIENT_ID_FIELD_NUMBER: builtins.int + OPERATION_FIELD_NUMBER: builtins.int + WARNINGS_FIELD_NUMBER: builtins.int + client_id: builtins.str + @property + def operation(self) -> global___ConfigResponse.Operation: + """(Required) The operation for the config.""" @property def warnings( self, @@ -976,27 +1154,16 @@ class ConfigResponse(google.protobuf.message.Message): self, *, client_id: builtins.str = ..., - keys: collections.abc.Iterable[builtins.str] | None = ..., - values: collections.abc.Iterable[builtins.str] | None = ..., - optional_values: collections.abc.Iterable[global___OptionalValue] | None = ..., - bools: collections.abc.Iterable[builtins.bool] | None = ..., + operation: global___ConfigResponse.Operation | None = ..., warnings: collections.abc.Iterable[builtins.str] | None = ..., ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["operation", b"operation"] + ) -> builtins.bool: ... def ClearField( self, field_name: typing_extensions.Literal[ - "bools", - b"bools", - "client_id", - b"client_id", - "keys", - b"keys", - "optional_values", - b"optional_values", - "values", - b"values", - "warnings", - b"warnings", + "client_id", b"client_id", "operation", b"operation", "warnings", b"warnings" ], ) -> None: ... From 4d8ff36b57d20f3612d8f8d964ab671c9d5cf743 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Feb 2023 14:56:45 -0800 Subject: [PATCH 07/14] Fix. --- python/pyspark/sql/connect/functions.py | 5 ----- .../tests/connect/test_parity_dataframe.py | 20 ------------------- 2 files changed, 25 deletions(-) diff --git a/python/pyspark/sql/connect/functions.py b/python/pyspark/sql/connect/functions.py index 7d61a86c8b5e7..87dfe90107d0e 100644 --- a/python/pyspark/sql/connect/functions.py +++ b/python/pyspark/sql/connect/functions.py @@ -2477,11 +2477,6 @@ def _test() -> None: # Spark Connect does not support Spark Context but the test depends on that. del pyspark.sql.connect.functions.monotonically_increasing_id.__doc__ - # TODO(SPARK-41834): implement Dataframe.conf - del pyspark.sql.connect.functions.from_unixtime.__doc__ - del pyspark.sql.connect.functions.timestamp_seconds.__doc__ - del pyspark.sql.connect.functions.unix_timestamp.__doc__ - # TODO(SPARK-41843): Implement SparkSession.udf del pyspark.sql.connect.functions.call_udf.__doc__ diff --git a/python/pyspark/sql/tests/connect/test_parity_dataframe.py b/python/pyspark/sql/tests/connect/test_parity_dataframe.py index 07cae0fb27d0d..60f6e78024604 100644 --- a/python/pyspark/sql/tests/connect/test_parity_dataframe.py +++ b/python/pyspark/sql/tests/connect/test_parity_dataframe.py @@ -46,11 +46,6 @@ def test_help_command(self): def test_invalid_join_method(self): super().test_invalid_join_method() - # TODO(SPARK-41834): Implement SparkSession.conf - @unittest.skip("Fails in Spark Connect, should enable.") - def test_join_without_on(self): - super().test_join_without_on() - # TODO(SPARK-41527): Implement DataFrame.observe @unittest.skip("Fails in Spark Connect, should enable.") def test_observe(self): @@ -75,11 +70,6 @@ def test_repartitionByRange_dataframe(self): def test_repr_behaviors(self): super().test_repr_behaviors() - # TODO(SPARK-41834): Implement SparkSession.conf - @unittest.skip("Fails in Spark Connect, should enable.") - def test_require_cross(self): - super().test_require_cross() - # TODO(SPARK-41874): Implement DataFrame `sameSemantics` @unittest.skip("Fails in Spark Connect, should enable.") def test_same_semantics_error(self): @@ -117,16 +107,6 @@ def test_to_pandas_for_array_of_struct(self): # Spark Connect's implementation is based on Arrow. super().check_to_pandas_for_array_of_struct(True) - # TODO(SPARK-41834): Implement SparkSession.conf - @unittest.skip("Fails in Spark Connect, should enable.") - def test_to_pandas_from_empty_dataframe(self): - super().test_to_pandas_from_empty_dataframe() - - # TODO(SPARK-41834): Implement SparkSession.conf - @unittest.skip("Fails in Spark Connect, should enable.") - def test_to_pandas_from_mixed_dataframe(self): - super().test_to_pandas_from_mixed_dataframe() - # TODO(SPARK-41834): Implement SparkSession.conf @unittest.skip("Fails in Spark Connect, should enable.") def test_to_pandas_from_null_dataframe(self): From d832e2bc09dd40d6b6baaf71c0e77c96cb3e9aee Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Feb 2023 15:01:03 -0800 Subject: [PATCH 08/14] Fix. --- .../connect/service/SparkConnectConfigHandler.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index d3fc1959407ea..3343c9d181d45 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -101,7 +101,8 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes val warnings = operation.getPairsList.asScala.flatMap { pair => val key = pair.getKey val value = SparkConnectConfigHandler.toOption(pair.getValue).orNull - response.addValues(SparkConnectConfigHandler.toOptionalValue(Option(conf.get(key, value)))) + response.addValues( + SparkConnectConfigHandler.toProtoOptionalValue(Option(conf.get(key, value)))) getWarning(key) } builder.setGetWithDefault(response.build()) @@ -114,7 +115,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes val builder = proto.ConfigResponse.Operation.newBuilder() val response = proto.ConfigResponse.GetOption.newBuilder() val warnings = operation.getKeysList.asScala.flatMap { key => - response.addValues(SparkConnectConfigHandler.toOptionalValue(conf.getOption(key))) + response.addValues(SparkConnectConfigHandler.toProtoOptionalValue(conf.getOption(key))) getWarning(key) } builder.setGetOption(response.build()) @@ -202,11 +203,9 @@ object SparkConnectConfigHandler { } } - def toOptionalValue(value: Option[String]): proto.OptionalValue = { + def toProtoOptionalValue(value: Option[String]): proto.OptionalValue = { val builder = proto.OptionalValue.newBuilder() - if (value.isDefined) { - builder.setValue(value.get) - } + value.foreach(builder.setValue(_)) builder.build() } } From 1b80142170a4229efcd5587879fa23f61466f05a Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Feb 2023 15:24:51 -0800 Subject: [PATCH 09/14] Fix. --- .../main/protobuf/spark/connect/base.proto | 15 +-- .../service/SparkConnectConfigHandler.scala | 15 --- python/pyspark/sql/connect/client.py | 6 -- python/pyspark/sql/connect/proto/base_pb2.py | 100 +++++++----------- python/pyspark/sql/connect/proto/base_pb2.pyi | 65 +----------- 5 files changed, 41 insertions(+), 160 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 30e2baff3832b..06d0bd0f60d55 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -222,8 +222,7 @@ message ConfigRequest { GetOption get_option = 4; GetAll get_all = 5; Unset unset = 6; - Contains contains = 7; - IsModifiable is_modifiable = 8; + IsModifiable is_modifiable = 7; } } @@ -257,11 +256,6 @@ message ConfigRequest { repeated string keys = 1; } - message Contains { - // (Required) The config keys to check if the config contains them. - repeated string keys = 1; - } - message IsModifiable { // (Required) The config keys to check the config is modifiable. repeated string keys = 1; @@ -288,8 +282,7 @@ message ConfigResponse { GetOption get_option = 4; GetAll get_all = 5; Unset unset = 6; - Contains contains = 7; - IsModifiable is_modifiable = 8; + IsModifiable is_modifiable = 7; } } @@ -315,10 +308,6 @@ message ConfigResponse { message Unset { } - message Contains { - repeated bool bools = 1; - } - message IsModifiable { repeated bool bools = 1; } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index 3343c9d181d45..9275293c16527 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -48,8 +48,6 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes handleGetAll(request.getOperation.getGetAll, session.conf) case proto.ConfigRequest.Operation.OpTypeCase.UNSET => handleUnset(request.getOperation.getUnset, session.conf) - case proto.ConfigRequest.Operation.OpTypeCase.CONTAINS => - handleContains(request.getOperation.getContains, session.conf) case proto.ConfigRequest.Operation.OpTypeCase.IS_MODIFIABLE => handleIsModifiable(request.getOperation.getIsModifiable, session.conf) case _ => throw new UnsupportedOperationException(s"${request.getOperation} not supported.") @@ -156,19 +154,6 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes (builder.build(), warnings) } - private def handleContains( - operation: proto.ConfigRequest.Contains, - conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { - val builder = proto.ConfigResponse.Operation.newBuilder() - val response = proto.ConfigResponse.Contains.newBuilder() - val warnings = operation.getKeysList.asScala.flatMap { key => - response.addBools(conf.contains(key)) - getWarning(key) - } - builder.setContains(response.build()) - (builder.build(), warnings) - } - private def handleIsModifiable( operation: proto.ConfigRequest.IsModifiable, conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { diff --git a/python/pyspark/sql/connect/client.py b/python/pyspark/sql/connect/client.py index be8c7471cd4bd..1169d24fc2c26 100644 --- a/python/pyspark/sql/connect/client.py +++ b/python/pyspark/sql/connect/client.py @@ -428,10 +428,6 @@ def __init__(self, pairs: List[Tuple[str, str]]): class Unset(Operation): pass - class Contains(Operation): - def __init__(self, bools: List[bool]): - self.bools = bools - class IsModifiable(Operation): def __init__(self, bools: List[bool]): self.bools = bools @@ -477,8 +473,6 @@ def fromProto(cls, pb: pb2.ConfigResponse) -> "ConfigResult": ) elif op_type == "unset": return ConfigResult(ConfigResult.Unset(), warnings) - elif op_type == "contains": - return ConfigResult(ConfigResult.Contains(list(pb.operation.contains.bools)), warnings) elif op_type == "is_modifiable": return ConfigResult( ConfigResult.IsModifiable(list(pb.operation.is_modifiable.bools)), warnings diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 17797cc122f46..d0545d11c91cf 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"2\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value"X\n\x10OptionalKeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\xb9\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x1a\xb7\x04\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12\x43\n\x08\x63ontains\x18\x07 \x01(\x0b\x32%.spark.connect.ConfigRequest.ContainsH\x00R\x08\x63ontains\x12P\n\ris_modifiable\x18\x08 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1aG\n\x0eGetWithDefault\x12\x35\n\x05pairs\x18\x01 \x03(\x0b\x32\x1f.spark.connect.OptionalKeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x1e\n\x08\x43ontains\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys"\x8d\x08\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\'.spark.connect.ConfigResponse.OperationR\toperation\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings\x1a\xbf\x04\n\tOperation\x12\x35\n\x03set\x18\x01 \x01(\x0b\x32!.spark.connect.ConfigResponse.SetH\x00R\x03set\x12\x35\n\x03get\x18\x02 \x01(\x0b\x32!.spark.connect.ConfigResponse.GetH\x00R\x03get\x12X\n\x10get_with_default\x18\x03 \x01(\x0b\x32,.spark.connect.ConfigResponse.GetWithDefaultH\x00R\x0egetWithDefault\x12H\n\nget_option\x18\x04 \x01(\x0b\x32\'.spark.connect.ConfigResponse.GetOptionH\x00R\tgetOption\x12?\n\x07get_all\x18\x05 \x01(\x0b\x32$.spark.connect.ConfigResponse.GetAllH\x00R\x06getAll\x12;\n\x05unset\x18\x06 \x01(\x0b\x32#.spark.connect.ConfigResponse.UnsetH\x00R\x05unset\x12\x44\n\x08\x63ontains\x18\x07 \x01(\x0b\x32&.spark.connect.ConfigResponse.ContainsH\x00R\x08\x63ontains\x12Q\n\ris_modifiable\x18\x08 \x01(\x0b\x32*.spark.connect.ConfigResponse.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x05\n\x03Set\x1a\x1d\n\x03Get\x12\x16\n\x06values\x18\x01 \x03(\tR\x06values\x1a\x46\n\x0eGetWithDefault\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x41\n\tGetOption\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x37\n\x06GetAll\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x07\n\x05Unset\x1a \n\x08\x43ontains\x12\x14\n\x05\x62ools\x18\x01 \x03(\x08R\x05\x62ools\x1a$\n\x0cIsModifiable\x12\x14\n\x05\x62ools\x18\x01 \x03(\x08R\x05\x62ools2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"2\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value"X\n\x10OptionalKeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\xd4\x07\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1aG\n\x0eGetWithDefault\x12\x35\n\x05pairs\x18\x01 \x03(\x0b\x32\x1f.spark.connect.OptionalKeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys"\xa5\x07\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\'.spark.connect.ConfigResponse.OperationR\toperation\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings\x1a\xf9\x03\n\tOperation\x12\x35\n\x03set\x18\x01 \x01(\x0b\x32!.spark.connect.ConfigResponse.SetH\x00R\x03set\x12\x35\n\x03get\x18\x02 \x01(\x0b\x32!.spark.connect.ConfigResponse.GetH\x00R\x03get\x12X\n\x10get_with_default\x18\x03 \x01(\x0b\x32,.spark.connect.ConfigResponse.GetWithDefaultH\x00R\x0egetWithDefault\x12H\n\nget_option\x18\x04 \x01(\x0b\x32\'.spark.connect.ConfigResponse.GetOptionH\x00R\tgetOption\x12?\n\x07get_all\x18\x05 \x01(\x0b\x32$.spark.connect.ConfigResponse.GetAllH\x00R\x06getAll\x12;\n\x05unset\x18\x06 \x01(\x0b\x32#.spark.connect.ConfigResponse.UnsetH\x00R\x05unset\x12Q\n\ris_modifiable\x18\x07 \x01(\x0b\x32*.spark.connect.ConfigResponse.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x05\n\x03Set\x1a\x1d\n\x03Get\x12\x16\n\x06values\x18\x01 \x03(\tR\x06values\x1a\x46\n\x0eGetWithDefault\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x41\n\tGetOption\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x37\n\x06GetAll\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x07\n\x05Unset\x1a$\n\x0cIsModifiable\x12\x14\n\x05\x62ools\x18\x01 \x03(\x08R\x05\x62ools2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -69,7 +69,6 @@ _CONFIGREQUEST_GETOPTION = _CONFIGREQUEST.nested_types_by_name["GetOption"] _CONFIGREQUEST_GETALL = _CONFIGREQUEST.nested_types_by_name["GetAll"] _CONFIGREQUEST_UNSET = _CONFIGREQUEST.nested_types_by_name["Unset"] -_CONFIGREQUEST_CONTAINS = _CONFIGREQUEST.nested_types_by_name["Contains"] _CONFIGREQUEST_ISMODIFIABLE = _CONFIGREQUEST.nested_types_by_name["IsModifiable"] _CONFIGRESPONSE = DESCRIPTOR.message_types_by_name["ConfigResponse"] _CONFIGRESPONSE_OPERATION = _CONFIGRESPONSE.nested_types_by_name["Operation"] @@ -79,7 +78,6 @@ _CONFIGRESPONSE_GETOPTION = _CONFIGRESPONSE.nested_types_by_name["GetOption"] _CONFIGRESPONSE_GETALL = _CONFIGRESPONSE.nested_types_by_name["GetAll"] _CONFIGRESPONSE_UNSET = _CONFIGRESPONSE.nested_types_by_name["Unset"] -_CONFIGRESPONSE_CONTAINS = _CONFIGRESPONSE.nested_types_by_name["Contains"] _CONFIGRESPONSE_ISMODIFIABLE = _CONFIGRESPONSE.nested_types_by_name["IsModifiable"] _EXPLAIN_EXPLAINMODE = _EXPLAIN.enum_types_by_name["ExplainMode"] Plan = _reflection.GeneratedProtocolMessageType( @@ -309,15 +307,6 @@ # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.Unset) }, ), - "Contains": _reflection.GeneratedProtocolMessageType( - "Contains", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGREQUEST_CONTAINS, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigRequest.Contains) - }, - ), "IsModifiable": _reflection.GeneratedProtocolMessageType( "IsModifiable", (_message.Message,), @@ -340,7 +329,6 @@ _sym_db.RegisterMessage(ConfigRequest.GetOption) _sym_db.RegisterMessage(ConfigRequest.GetAll) _sym_db.RegisterMessage(ConfigRequest.Unset) -_sym_db.RegisterMessage(ConfigRequest.Contains) _sym_db.RegisterMessage(ConfigRequest.IsModifiable) ConfigResponse = _reflection.GeneratedProtocolMessageType( @@ -410,15 +398,6 @@ # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Unset) }, ), - "Contains": _reflection.GeneratedProtocolMessageType( - "Contains", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_CONTAINS, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Contains) - }, - ), "IsModifiable": _reflection.GeneratedProtocolMessageType( "IsModifiable", (_message.Message,), @@ -441,7 +420,6 @@ _sym_db.RegisterMessage(ConfigResponse.GetOption) _sym_db.RegisterMessage(ConfigResponse.GetAll) _sym_db.RegisterMessage(ConfigResponse.Unset) -_sym_db.RegisterMessage(ConfigResponse.Contains) _sym_db.RegisterMessage(ConfigResponse.IsModifiable) _SPARKCONNECTSERVICE = DESCRIPTOR.services_by_name["SparkConnectService"] @@ -484,45 +462,41 @@ _OPTIONALKEYVALUE._serialized_start = 2215 _OPTIONALKEYVALUE._serialized_end = 2303 _CONFIGREQUEST._serialized_start = 2306 - _CONFIGREQUEST._serialized_end = 3387 + _CONFIGREQUEST._serialized_end = 3286 _CONFIGREQUEST_OPERATION._serialized_start = 2486 - _CONFIGREQUEST_OPERATION._serialized_end = 3053 - _CONFIGREQUEST_SET._serialized_start = 3055 - _CONFIGREQUEST_SET._serialized_end = 3107 - _CONFIGREQUEST_GET._serialized_start = 3109 - _CONFIGREQUEST_GET._serialized_end = 3134 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 3136 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3207 - _CONFIGREQUEST_GETOPTION._serialized_start = 3209 - _CONFIGREQUEST_GETOPTION._serialized_end = 3240 - _CONFIGREQUEST_GETALL._serialized_start = 3242 - _CONFIGREQUEST_GETALL._serialized_end = 3290 - _CONFIGREQUEST_UNSET._serialized_start = 3292 - _CONFIGREQUEST_UNSET._serialized_end = 3319 - _CONFIGREQUEST_CONTAINS._serialized_start = 3321 - _CONFIGREQUEST_CONTAINS._serialized_end = 3351 - _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3353 - _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3387 - _CONFIGRESPONSE._serialized_start = 3390 - _CONFIGRESPONSE._serialized_end = 4427 - _CONFIGRESPONSE_OPERATION._serialized_start = 3537 - _CONFIGRESPONSE_OPERATION._serialized_end = 4112 - _CONFIGRESPONSE_SET._serialized_start = 3055 - _CONFIGRESPONSE_SET._serialized_end = 3060 - _CONFIGRESPONSE_GET._serialized_start = 4121 - _CONFIGRESPONSE_GET._serialized_end = 4150 - _CONFIGRESPONSE_GETWITHDEFAULT._serialized_start = 4152 - _CONFIGRESPONSE_GETWITHDEFAULT._serialized_end = 4222 - _CONFIGRESPONSE_GETOPTION._serialized_start = 4224 - _CONFIGRESPONSE_GETOPTION._serialized_end = 4289 - _CONFIGRESPONSE_GETALL._serialized_start = 4291 - _CONFIGRESPONSE_GETALL._serialized_end = 4346 - _CONFIGRESPONSE_UNSET._serialized_start = 3292 - _CONFIGRESPONSE_UNSET._serialized_end = 3299 - _CONFIGRESPONSE_CONTAINS._serialized_start = 4357 - _CONFIGRESPONSE_CONTAINS._serialized_end = 4389 - _CONFIGRESPONSE_ISMODIFIABLE._serialized_start = 4391 - _CONFIGRESPONSE_ISMODIFIABLE._serialized_end = 4427 - _SPARKCONNECTSERVICE._serialized_start = 4430 - _SPARKCONNECTSERVICE._serialized_end = 4702 + _CONFIGREQUEST_OPERATION._serialized_end = 2984 + _CONFIGREQUEST_SET._serialized_start = 2986 + _CONFIGREQUEST_SET._serialized_end = 3038 + _CONFIGREQUEST_GET._serialized_start = 3040 + _CONFIGREQUEST_GET._serialized_end = 3065 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 3067 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3138 + _CONFIGREQUEST_GETOPTION._serialized_start = 3140 + _CONFIGREQUEST_GETOPTION._serialized_end = 3171 + _CONFIGREQUEST_GETALL._serialized_start = 3173 + _CONFIGREQUEST_GETALL._serialized_end = 3221 + _CONFIGREQUEST_UNSET._serialized_start = 3223 + _CONFIGREQUEST_UNSET._serialized_end = 3250 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3252 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3286 + _CONFIGRESPONSE._serialized_start = 3289 + _CONFIGRESPONSE._serialized_end = 4222 + _CONFIGRESPONSE_OPERATION._serialized_start = 3436 + _CONFIGRESPONSE_OPERATION._serialized_end = 3941 + _CONFIGRESPONSE_SET._serialized_start = 2986 + _CONFIGRESPONSE_SET._serialized_end = 2991 + _CONFIGRESPONSE_GET._serialized_start = 3950 + _CONFIGRESPONSE_GET._serialized_end = 3979 + _CONFIGRESPONSE_GETWITHDEFAULT._serialized_start = 3981 + _CONFIGRESPONSE_GETWITHDEFAULT._serialized_end = 4051 + _CONFIGRESPONSE_GETOPTION._serialized_start = 4053 + _CONFIGRESPONSE_GETOPTION._serialized_end = 4118 + _CONFIGRESPONSE_GETALL._serialized_start = 4120 + _CONFIGRESPONSE_GETALL._serialized_end = 4175 + _CONFIGRESPONSE_UNSET._serialized_start = 3223 + _CONFIGRESPONSE_UNSET._serialized_end = 3230 + _CONFIGRESPONSE_ISMODIFIABLE._serialized_start = 4186 + _CONFIGRESPONSE_ISMODIFIABLE._serialized_end = 4222 + _SPARKCONNECTSERVICE._serialized_start = 4225 + _SPARKCONNECTSERVICE._serialized_end = 4497 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 40918e0b66de1..6cad23292f24c 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -655,7 +655,6 @@ class ConfigRequest(google.protobuf.message.Message): GET_OPTION_FIELD_NUMBER: builtins.int GET_ALL_FIELD_NUMBER: builtins.int UNSET_FIELD_NUMBER: builtins.int - CONTAINS_FIELD_NUMBER: builtins.int IS_MODIFIABLE_FIELD_NUMBER: builtins.int @property def set(self) -> global___ConfigRequest.Set: ... @@ -670,8 +669,6 @@ class ConfigRequest(google.protobuf.message.Message): @property def unset(self) -> global___ConfigRequest.Unset: ... @property - def contains(self) -> global___ConfigRequest.Contains: ... - @property def is_modifiable(self) -> global___ConfigRequest.IsModifiable: ... def __init__( self, @@ -682,14 +679,11 @@ class ConfigRequest(google.protobuf.message.Message): get_option: global___ConfigRequest.GetOption | None = ..., get_all: global___ConfigRequest.GetAll | None = ..., unset: global___ConfigRequest.Unset | None = ..., - contains: global___ConfigRequest.Contains | None = ..., is_modifiable: global___ConfigRequest.IsModifiable | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ - "contains", - b"contains", "get", b"get", "get_all", @@ -711,8 +705,6 @@ class ConfigRequest(google.protobuf.message.Message): def ClearField( self, field_name: typing_extensions.Literal[ - "contains", - b"contains", "get", b"get", "get_all", @@ -734,14 +726,7 @@ class ConfigRequest(google.protobuf.message.Message): def WhichOneof( self, oneof_group: typing_extensions.Literal["op_type", b"op_type"] ) -> typing_extensions.Literal[ - "set", - "get", - "get_with_default", - "get_option", - "get_all", - "unset", - "contains", - "is_modifiable", + "set", "get", "get_with_default", "get_option", "get_all", "unset", "is_modifiable" ] | None: ... class Set(google.protobuf.message.Message): @@ -847,22 +832,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... - class Contains(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - KEYS_FIELD_NUMBER: builtins.int - @property - def keys( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: - """(Required) The config keys to check if the config contains them.""" - def __init__( - self, - *, - keys: collections.abc.Iterable[builtins.str] | None = ..., - ) -> None: ... - def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... - class IsModifiable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -930,7 +899,6 @@ class ConfigResponse(google.protobuf.message.Message): GET_OPTION_FIELD_NUMBER: builtins.int GET_ALL_FIELD_NUMBER: builtins.int UNSET_FIELD_NUMBER: builtins.int - CONTAINS_FIELD_NUMBER: builtins.int IS_MODIFIABLE_FIELD_NUMBER: builtins.int @property def set(self) -> global___ConfigResponse.Set: ... @@ -945,8 +913,6 @@ class ConfigResponse(google.protobuf.message.Message): @property def unset(self) -> global___ConfigResponse.Unset: ... @property - def contains(self) -> global___ConfigResponse.Contains: ... - @property def is_modifiable(self) -> global___ConfigResponse.IsModifiable: ... def __init__( self, @@ -957,14 +923,11 @@ class ConfigResponse(google.protobuf.message.Message): get_option: global___ConfigResponse.GetOption | None = ..., get_all: global___ConfigResponse.GetAll | None = ..., unset: global___ConfigResponse.Unset | None = ..., - contains: global___ConfigResponse.Contains | None = ..., is_modifiable: global___ConfigResponse.IsModifiable | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ - "contains", - b"contains", "get", b"get", "get_all", @@ -986,8 +949,6 @@ class ConfigResponse(google.protobuf.message.Message): def ClearField( self, field_name: typing_extensions.Literal[ - "contains", - b"contains", "get", b"get", "get_all", @@ -1009,14 +970,7 @@ class ConfigResponse(google.protobuf.message.Message): def WhichOneof( self, oneof_group: typing_extensions.Literal["op_type", b"op_type"] ) -> typing_extensions.Literal[ - "set", - "get", - "get_with_default", - "get_option", - "get_all", - "unset", - "contains", - "is_modifiable", + "set", "get", "get_with_default", "get_option", "get_all", "unset", "is_modifiable" ] | None: ... class Set(google.protobuf.message.Message): @@ -1105,21 +1059,6 @@ class ConfigResponse(google.protobuf.message.Message): self, ) -> None: ... - class Contains(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - BOOLS_FIELD_NUMBER: builtins.int - @property - def bools( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.bool]: ... - def __init__( - self, - *, - bools: collections.abc.Iterable[builtins.bool] | None = ..., - ) -> None: ... - def ClearField(self, field_name: typing_extensions.Literal["bools", b"bools"]) -> None: ... - class IsModifiable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor From 88fb676f619981a4bd795719b0251b1e7e43e7f2 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Feb 2023 15:52:48 -0800 Subject: [PATCH 10/14] Fix. --- .../connect/service/SparkConnectConfigHandler.scala | 12 ++++++------ .../pyspark/sql/tests/connect/test_connect_basic.py | 1 - 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index 9275293c16527..256b6c883deef 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -73,7 +73,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes val value = pair.getValue conf.set(key, value) getWarning(key) - } + }.toSeq builder.setSet(response.build()) (builder.build(), warnings) } @@ -86,7 +86,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes val warnings = operation.getKeysList.asScala.flatMap { key => response.addValues(conf.get(key)) getWarning(key) - } + }.toSeq builder.setGet(response.build()) (builder.build(), warnings) } @@ -102,7 +102,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes response.addValues( SparkConnectConfigHandler.toProtoOptionalValue(Option(conf.get(key, value)))) getWarning(key) - } + }.toSeq builder.setGetWithDefault(response.build()) (builder.build(), warnings) } @@ -115,7 +115,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes val warnings = operation.getKeysList.asScala.flatMap { key => response.addValues(SparkConnectConfigHandler.toProtoOptionalValue(conf.getOption(key))) getWarning(key) - } + }.toSeq builder.setGetOption(response.build()) (builder.build(), warnings) } @@ -149,7 +149,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes val warnings = operation.getKeysList.asScala.flatMap { key => conf.unset(key) getWarning(key) - } + }.toSeq builder.setUnset(response.build()) (builder.build(), warnings) } @@ -162,7 +162,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes val warnings = operation.getKeysList.asScala.flatMap { key => response.addBools(conf.isModifiable(key)) getWarning(key) - } + }.toSeq builder.setIsModifiable(response.build()) (builder.build(), warnings) } diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index adcd457a10515..84c3e4f23a6ab 100644 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -2796,7 +2796,6 @@ def test_unsupported_session_functions(self): for f in ( "newSession", - "conf", "sparkContext", "streams", "readStream", From 34b18b28989c522767b7dc96436f4709103e0573 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Feb 2023 17:27:47 -0800 Subject: [PATCH 11/14] Fix. --- .../main/protobuf/spark/connect/base.proto | 5 ++ python/pyspark/sql/connect/client.py | 1 + python/pyspark/sql/connect/proto/base_pb2.py | 76 +++++++++---------- python/pyspark/sql/connect/proto/base_pb2.pyi | 30 +++++++- 4 files changed, 72 insertions(+), 40 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 06d0bd0f60d55..d87809da3ab06 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -214,6 +214,11 @@ message ConfigRequest { // (Required) The operation for the config. Operation operation = 3; + // Provides optional information about the client sending the request. This field + // can be used for language or version specific information and is only intended for + // logging purposes and will not be interpreted by the server. + optional string client_type = 4; + message Operation { oneof op_type { Set set = 1; diff --git a/python/pyspark/sql/connect/client.py b/python/pyspark/sql/connect/client.py index 1169d24fc2c26..8019c01bab85e 100644 --- a/python/pyspark/sql/connect/client.py +++ b/python/pyspark/sql/connect/client.py @@ -818,6 +818,7 @@ def _execute_and_fetch( def _config_request_with_metadata(self) -> pb2.ConfigRequest: req = pb2.ConfigRequest() req.client_id = self._session_id + req.client_type = self._builder.userAgent if self._user_id: req.user_context.user_id = self._user_id return req diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index d0545d11c91cf..93bf4e3bd7a80 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"2\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value"X\n\x10OptionalKeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\xd4\x07\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1aG\n\x0eGetWithDefault\x12\x35\n\x05pairs\x18\x01 \x03(\x0b\x32\x1f.spark.connect.OptionalKeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys"\xa5\x07\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\'.spark.connect.ConfigResponse.OperationR\toperation\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings\x1a\xf9\x03\n\tOperation\x12\x35\n\x03set\x18\x01 \x01(\x0b\x32!.spark.connect.ConfigResponse.SetH\x00R\x03set\x12\x35\n\x03get\x18\x02 \x01(\x0b\x32!.spark.connect.ConfigResponse.GetH\x00R\x03get\x12X\n\x10get_with_default\x18\x03 \x01(\x0b\x32,.spark.connect.ConfigResponse.GetWithDefaultH\x00R\x0egetWithDefault\x12H\n\nget_option\x18\x04 \x01(\x0b\x32\'.spark.connect.ConfigResponse.GetOptionH\x00R\tgetOption\x12?\n\x07get_all\x18\x05 \x01(\x0b\x32$.spark.connect.ConfigResponse.GetAllH\x00R\x06getAll\x12;\n\x05unset\x18\x06 \x01(\x0b\x32#.spark.connect.ConfigResponse.UnsetH\x00R\x05unset\x12Q\n\ris_modifiable\x18\x07 \x01(\x0b\x32*.spark.connect.ConfigResponse.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x05\n\x03Set\x1a\x1d\n\x03Get\x12\x16\n\x06values\x18\x01 \x03(\tR\x06values\x1a\x46\n\x0eGetWithDefault\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x41\n\tGetOption\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x37\n\x06GetAll\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x07\n\x05Unset\x1a$\n\x0cIsModifiable\x12\x14\n\x05\x62ools\x18\x01 \x03(\x08R\x05\x62ools2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"2\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value"X\n\x10OptionalKeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\x8a\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1aG\n\x0eGetWithDefault\x12\x35\n\x05pairs\x18\x01 \x03(\x0b\x32\x1f.spark.connect.OptionalKeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB\x0e\n\x0c_client_type"\xa5\x07\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\'.spark.connect.ConfigResponse.OperationR\toperation\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings\x1a\xf9\x03\n\tOperation\x12\x35\n\x03set\x18\x01 \x01(\x0b\x32!.spark.connect.ConfigResponse.SetH\x00R\x03set\x12\x35\n\x03get\x18\x02 \x01(\x0b\x32!.spark.connect.ConfigResponse.GetH\x00R\x03get\x12X\n\x10get_with_default\x18\x03 \x01(\x0b\x32,.spark.connect.ConfigResponse.GetWithDefaultH\x00R\x0egetWithDefault\x12H\n\nget_option\x18\x04 \x01(\x0b\x32\'.spark.connect.ConfigResponse.GetOptionH\x00R\tgetOption\x12?\n\x07get_all\x18\x05 \x01(\x0b\x32$.spark.connect.ConfigResponse.GetAllH\x00R\x06getAll\x12;\n\x05unset\x18\x06 \x01(\x0b\x32#.spark.connect.ConfigResponse.UnsetH\x00R\x05unset\x12Q\n\ris_modifiable\x18\x07 \x01(\x0b\x32*.spark.connect.ConfigResponse.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x05\n\x03Set\x1a\x1d\n\x03Get\x12\x16\n\x06values\x18\x01 \x03(\tR\x06values\x1a\x46\n\x0eGetWithDefault\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x41\n\tGetOption\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x37\n\x06GetAll\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x07\n\x05Unset\x1a$\n\x0cIsModifiable\x12\x14\n\x05\x62ools\x18\x01 \x03(\x08R\x05\x62ools2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -462,41 +462,41 @@ _OPTIONALKEYVALUE._serialized_start = 2215 _OPTIONALKEYVALUE._serialized_end = 2303 _CONFIGREQUEST._serialized_start = 2306 - _CONFIGREQUEST._serialized_end = 3286 - _CONFIGREQUEST_OPERATION._serialized_start = 2486 - _CONFIGREQUEST_OPERATION._serialized_end = 2984 - _CONFIGREQUEST_SET._serialized_start = 2986 - _CONFIGREQUEST_SET._serialized_end = 3038 - _CONFIGREQUEST_GET._serialized_start = 3040 - _CONFIGREQUEST_GET._serialized_end = 3065 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 3067 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3138 - _CONFIGREQUEST_GETOPTION._serialized_start = 3140 - _CONFIGREQUEST_GETOPTION._serialized_end = 3171 - _CONFIGREQUEST_GETALL._serialized_start = 3173 - _CONFIGREQUEST_GETALL._serialized_end = 3221 - _CONFIGREQUEST_UNSET._serialized_start = 3223 - _CONFIGREQUEST_UNSET._serialized_end = 3250 - _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3252 - _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3286 - _CONFIGRESPONSE._serialized_start = 3289 - _CONFIGRESPONSE._serialized_end = 4222 - _CONFIGRESPONSE_OPERATION._serialized_start = 3436 - _CONFIGRESPONSE_OPERATION._serialized_end = 3941 - _CONFIGRESPONSE_SET._serialized_start = 2986 - _CONFIGRESPONSE_SET._serialized_end = 2991 - _CONFIGRESPONSE_GET._serialized_start = 3950 - _CONFIGRESPONSE_GET._serialized_end = 3979 - _CONFIGRESPONSE_GETWITHDEFAULT._serialized_start = 3981 - _CONFIGRESPONSE_GETWITHDEFAULT._serialized_end = 4051 - _CONFIGRESPONSE_GETOPTION._serialized_start = 4053 - _CONFIGRESPONSE_GETOPTION._serialized_end = 4118 - _CONFIGRESPONSE_GETALL._serialized_start = 4120 - _CONFIGRESPONSE_GETALL._serialized_end = 4175 - _CONFIGRESPONSE_UNSET._serialized_start = 3223 - _CONFIGRESPONSE_UNSET._serialized_end = 3230 - _CONFIGRESPONSE_ISMODIFIABLE._serialized_start = 4186 - _CONFIGRESPONSE_ISMODIFIABLE._serialized_end = 4222 - _SPARKCONNECTSERVICE._serialized_start = 4225 - _SPARKCONNECTSERVICE._serialized_end = 4497 + _CONFIGREQUEST._serialized_end = 3340 + _CONFIGREQUEST_OPERATION._serialized_start = 2524 + _CONFIGREQUEST_OPERATION._serialized_end = 3022 + _CONFIGREQUEST_SET._serialized_start = 3024 + _CONFIGREQUEST_SET._serialized_end = 3076 + _CONFIGREQUEST_GET._serialized_start = 3078 + _CONFIGREQUEST_GET._serialized_end = 3103 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 3105 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3176 + _CONFIGREQUEST_GETOPTION._serialized_start = 3178 + _CONFIGREQUEST_GETOPTION._serialized_end = 3209 + _CONFIGREQUEST_GETALL._serialized_start = 3211 + _CONFIGREQUEST_GETALL._serialized_end = 3259 + _CONFIGREQUEST_UNSET._serialized_start = 3261 + _CONFIGREQUEST_UNSET._serialized_end = 3288 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3290 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3324 + _CONFIGRESPONSE._serialized_start = 3343 + _CONFIGRESPONSE._serialized_end = 4276 + _CONFIGRESPONSE_OPERATION._serialized_start = 3490 + _CONFIGRESPONSE_OPERATION._serialized_end = 3995 + _CONFIGRESPONSE_SET._serialized_start = 3024 + _CONFIGRESPONSE_SET._serialized_end = 3029 + _CONFIGRESPONSE_GET._serialized_start = 4004 + _CONFIGRESPONSE_GET._serialized_end = 4033 + _CONFIGRESPONSE_GETWITHDEFAULT._serialized_start = 4035 + _CONFIGRESPONSE_GETWITHDEFAULT._serialized_end = 4105 + _CONFIGRESPONSE_GETOPTION._serialized_start = 4107 + _CONFIGRESPONSE_GETOPTION._serialized_end = 4172 + _CONFIGRESPONSE_GETALL._serialized_start = 4174 + _CONFIGRESPONSE_GETALL._serialized_end = 4229 + _CONFIGRESPONSE_UNSET._serialized_start = 3261 + _CONFIGRESPONSE_UNSET._serialized_end = 3268 + _CONFIGRESPONSE_ISMODIFIABLE._serialized_start = 4240 + _CONFIGRESPONSE_ISMODIFIABLE._serialized_end = 4276 + _SPARKCONNECTSERVICE._serialized_start = 4279 + _SPARKCONNECTSERVICE._serialized_end = 4551 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 6cad23292f24c..c5ae189add4fa 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -851,6 +851,7 @@ class ConfigRequest(google.protobuf.message.Message): CLIENT_ID_FIELD_NUMBER: builtins.int USER_CONTEXT_FIELD_NUMBER: builtins.int OPERATION_FIELD_NUMBER: builtins.int + CLIENT_TYPE_FIELD_NUMBER: builtins.int client_id: builtins.str """(Required) @@ -863,25 +864,50 @@ class ConfigRequest(google.protobuf.message.Message): @property def operation(self) -> global___ConfigRequest.Operation: """(Required) The operation for the config.""" + client_type: builtins.str + """Provides optional information about the client sending the request. This field + can be used for language or version specific information and is only intended for + logging purposes and will not be interpreted by the server. + """ def __init__( self, *, client_id: builtins.str = ..., user_context: global___UserContext | None = ..., operation: global___ConfigRequest.Operation | None = ..., + client_type: builtins.str | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ - "operation", b"operation", "user_context", b"user_context" + "_client_type", + b"_client_type", + "client_type", + b"client_type", + "operation", + b"operation", + "user_context", + b"user_context", ], ) -> builtins.bool: ... def ClearField( self, field_name: typing_extensions.Literal[ - "client_id", b"client_id", "operation", b"operation", "user_context", b"user_context" + "_client_type", + b"_client_type", + "client_id", + b"client_id", + "client_type", + b"client_type", + "operation", + b"operation", + "user_context", + b"user_context", ], ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_client_type", b"_client_type"] + ) -> typing_extensions.Literal["client_type"] | None: ... global___ConfigRequest = ConfigRequest From d7dff30101ba7c5d9c8a179c5ac6702dd6a08dfd Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 23 Feb 2023 17:35:11 -0800 Subject: [PATCH 12/14] Fix. --- python/pyspark/sql/connect/session.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 5c685df91df6e..c95279a8c8e4c 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -422,7 +422,7 @@ def newSession(self) -> Any: raise NotImplementedError("newSession() is not implemented.") @property - def conf(self) -> Any: + def conf(self) -> RuntimeConf: return RuntimeConf(self.client) @property From 6f4743a43b8004dd44f6bf1e14fd3842ea8d0c56 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 24 Feb 2023 11:15:40 -0800 Subject: [PATCH 13/14] Fix. --- .../main/protobuf/spark/connect/base.proto | 53 +--- .../service/SparkConnectConfigHandler.scala | 137 +++++------ python/pyspark/sql/conf.py | 21 +- python/pyspark/sql/connect/client.py | 81 +----- python/pyspark/sql/connect/conf.py | 37 ++- python/pyspark/sql/connect/proto/base_pb2.py | 166 ++----------- python/pyspark/sql/connect/proto/base_pb2.pyi | 232 ++---------------- python/pyspark/sql/tests/test_conf.py | 5 + 8 files changed, 155 insertions(+), 577 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index d87809da3ab06..4cd6dca23a43d 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -190,12 +190,6 @@ message OptionalValue { // The key-value pair for the config request and response. message KeyValue { - string key = 1; - string value = 2; -} - -// The key-value pair for the config request and response when the value can be optional. -message OptionalKeyValue { string key = 1; OptionalValue value = 2; } @@ -243,7 +237,7 @@ message ConfigRequest { message GetWithDefault { // (Required) The config key-value paris to get. The value will be used as the default value. - repeated OptionalKeyValue pairs = 1; + repeated KeyValue pairs = 1; } message GetOption { @@ -271,51 +265,16 @@ message ConfigRequest { message ConfigResponse { string client_id = 1; - // (Required) The operation for the config. - Operation operation = 2; + // (Optional) The result key-value pairs. + // + // Available when the operation is 'Get', 'GetWithDefault', 'GetOption', 'GetAll'. + // Also available for the operation 'IsModifiable' with boolean string "true" and "false". + repeated KeyValue pairs = 2; // (Optional) // // Warning messages for deprecated or unsupported configurations. repeated string warnings = 3; - - message Operation { - oneof op_type { - Set set = 1; - Get get = 2; - GetWithDefault get_with_default = 3; - GetOption get_option = 4; - GetAll get_all = 5; - Unset unset = 6; - IsModifiable is_modifiable = 7; - } - } - - message Set { - } - - message Get { - repeated string values = 1; - } - - message GetWithDefault { - repeated OptionalValue values = 1; - } - - message GetOption { - repeated OptionalValue values = 1; - } - - message GetAll { - repeated KeyValue pairs = 1; - } - - message Unset { - } - - message IsModifiable { - repeated bool bools = 1; - } } // Main interface for the SparkConnect service. diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index 256b6c883deef..92a506934e220 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -35,7 +35,7 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes .getOrCreateIsolatedSession(request.getUserContext.getUserId, request.getClientId) .session - val (operation, warnings) = request.getOperation.getOpTypeCase match { + val builder = request.getOperation.getOpTypeCase match { case proto.ConfigRequest.Operation.OpTypeCase.SET => handleSet(request.getOperation.getSet, session.conf) case proto.ConfigRequest.Operation.OpTypeCase.GET => @@ -53,78 +53,66 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes case _ => throw new UnsupportedOperationException(s"${request.getOperation} not supported.") } - val response = proto.ConfigResponse - .newBuilder() - .setClientId(request.getClientId) - .setOperation(operation) - .addAllWarnings(warnings.asJava) - .build() - responseObserver.onNext(response) + builder.setClientId(request.getClientId) + responseObserver.onNext(builder.build()) responseObserver.onCompleted() } private def handleSet( operation: proto.ConfigRequest.Set, - conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { - val builder = proto.ConfigResponse.Operation.newBuilder() - val response = proto.ConfigResponse.Set.newBuilder() - val warnings = operation.getPairsList.asScala.flatMap { pair => + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getPairsList.asScala.iterator.foreach { pair => val key = pair.getKey - val value = pair.getValue + val value = SparkConnectConfigHandler.toOption(pair.getValue).orNull conf.set(key, value) - getWarning(key) - }.toSeq - builder.setSet(response.build()) - (builder.build(), warnings) + getWarning(key).foreach(builder.addWarnings) + } + builder } private def handleGet( operation: proto.ConfigRequest.Get, - conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { - val builder = proto.ConfigResponse.Operation.newBuilder() - val response = proto.ConfigResponse.Get.newBuilder() - val warnings = operation.getKeysList.asScala.flatMap { key => - response.addValues(conf.get(key)) - getWarning(key) - }.toSeq - builder.setGet(response.build()) - (builder.build(), warnings) + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => + val value = conf.get(key) + builder.addPairs(SparkConnectConfigHandler.toProtoKeyValue(key, Option(value))) + getWarning(key).foreach(builder.addWarnings) + } + builder } private def handleGetWithDefault( operation: proto.ConfigRequest.GetWithDefault, - conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { - val builder = proto.ConfigResponse.Operation.newBuilder() - val response = proto.ConfigResponse.GetWithDefault.newBuilder() - val warnings = operation.getPairsList.asScala.flatMap { pair => + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getPairsList.asScala.iterator.foreach { pair => val key = pair.getKey - val value = SparkConnectConfigHandler.toOption(pair.getValue).orNull - response.addValues( - SparkConnectConfigHandler.toProtoOptionalValue(Option(conf.get(key, value)))) - getWarning(key) - }.toSeq - builder.setGetWithDefault(response.build()) - (builder.build(), warnings) + val default = SparkConnectConfigHandler.toOption(pair.getValue).orNull + val value = conf.get(key, default) + builder.addPairs(SparkConnectConfigHandler.toProtoKeyValue(key, Option(value))) + getWarning(key).foreach(builder.addWarnings) + } + builder } private def handleGetOption( operation: proto.ConfigRequest.GetOption, - conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { - val builder = proto.ConfigResponse.Operation.newBuilder() - val response = proto.ConfigResponse.GetOption.newBuilder() - val warnings = operation.getKeysList.asScala.flatMap { key => - response.addValues(SparkConnectConfigHandler.toProtoOptionalValue(conf.getOption(key))) - getWarning(key) - }.toSeq - builder.setGetOption(response.build()) - (builder.build(), warnings) + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => + val value = conf.getOption(key) + builder.addPairs(SparkConnectConfigHandler.toProtoKeyValue(key, value)) + getWarning(key).foreach(builder.addWarnings) + } + builder } private def handleGetAll( operation: proto.ConfigRequest.GetAll, - conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { - val builder = proto.ConfigResponse.Operation.newBuilder() - val response = proto.ConfigResponse.GetAll.newBuilder() + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() val results = if (operation.hasPrefix) { val prefix = operation.getPrefix conf.getAll.iterator @@ -133,38 +121,34 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes } else { conf.getAll.iterator } - val warnings = results.flatMap { case (key, value) => - response.addPairs(proto.KeyValue.newBuilder().setKey(key).setValue(value).build()) - getWarning(key) - }.toSeq - builder.setGetAll(response.build()) - (builder.build(), warnings) + results.foreach { case (key, value) => + builder.addPairs(SparkConnectConfigHandler.toProtoKeyValue(key, Option(value))) + getWarning(key).foreach(builder.addWarnings) + } + builder } private def handleUnset( operation: proto.ConfigRequest.Unset, - conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { - val builder = proto.ConfigResponse.Operation.newBuilder() - val response = proto.ConfigResponse.Unset.newBuilder() - val warnings = operation.getKeysList.asScala.flatMap { key => + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => conf.unset(key) - getWarning(key) - }.toSeq - builder.setUnset(response.build()) - (builder.build(), warnings) + getWarning(key).foreach(builder.addWarnings) + } + builder } private def handleIsModifiable( operation: proto.ConfigRequest.IsModifiable, - conf: RuntimeConfig): (proto.ConfigResponse.Operation, Seq[String]) = { - val builder = proto.ConfigResponse.Operation.newBuilder() - val response = proto.ConfigResponse.IsModifiable.newBuilder() - val warnings = operation.getKeysList.asScala.flatMap { key => - response.addBools(conf.isModifiable(key)) - getWarning(key) - }.toSeq - builder.setIsModifiable(response.build()) - (builder.build(), warnings) + conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val builder = proto.ConfigResponse.newBuilder() + operation.getKeysList.asScala.iterator.foreach { key => + val value = conf.isModifiable(key) + builder.addPairs(SparkConnectConfigHandler.toProtoKeyValue(key, Option(value.toString))) + getWarning(key).foreach(builder.addWarnings) + } + builder } private def getWarning(key: String): Option[String] = { @@ -190,7 +174,14 @@ object SparkConnectConfigHandler { def toProtoOptionalValue(value: Option[String]): proto.OptionalValue = { val builder = proto.OptionalValue.newBuilder() - value.foreach(builder.setValue(_)) + value.foreach(builder.setValue) + builder.build() + } + + def toProtoKeyValue(key: String, value: Option[String]): proto.KeyValue = { + val builder = proto.KeyValue.newBuilder() + builder.setKey(key) + builder.setValue(toProtoOptionalValue(value)) builder.build() } } diff --git a/python/pyspark/sql/conf.py b/python/pyspark/sql/conf.py index 01e77b10fc6ae..e8b258c9bf826 100644 --- a/python/pyspark/sql/conf.py +++ b/python/pyspark/sql/conf.py @@ -28,6 +28,9 @@ class RuntimeConfig: """User-facing configuration API, accessible through `SparkSession.conf`. Options set here are automatically propagated to the Hadoop configuration during I/O. + + .. versionchanged:: 3.4.0 + Support Spark Connect. """ def __init__(self, jconf: JavaObject) -> None: @@ -36,7 +39,11 @@ def __init__(self, jconf: JavaObject) -> None: @since(2.0) def set(self, key: str, value: Union[str, int, bool]) -> None: - """Sets the given Spark runtime configuration property.""" + """Sets the given Spark runtime configuration property. + + .. versionchanged:: 3.4.0 + Support Spark Connect. + """ self._jconf.set(key, value) @since(2.0) @@ -45,6 +52,9 @@ def get( ) -> Optional[str]: """Returns the value of Spark runtime configuration property for the given key, assuming it is set. + + .. versionchanged:: 3.4.0 + Support Spark Connect. """ self._checkType(key, "key") if default is _NoValue: @@ -56,7 +66,11 @@ def get( @since(2.0) def unset(self, key: str) -> None: - """Resets the configuration property for the given key.""" + """Resets the configuration property for the given key. + + .. versionchanged:: 3.4.0 + Support Spark Connect. + """ self._jconf.unset(key) def _checkType(self, obj: Any, identifier: str) -> None: @@ -70,6 +84,9 @@ def _checkType(self, obj: Any, identifier: str) -> None: def isModifiable(self, key: str) -> bool: """Indicates whether the configuration property with the given key is modifiable in the current session. + + .. versionchanged:: 3.4.0 + Support Spark Connect. """ return self._jconf.isModifiable(key) diff --git a/python/pyspark/sql/connect/client.py b/python/pyspark/sql/connect/client.py index 8019c01bab85e..128020953a759 100644 --- a/python/pyspark/sql/connect/client.py +++ b/python/pyspark/sql/connect/client.py @@ -403,82 +403,19 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": class ConfigResult: - class Operation: - pass - - class Set(Operation): - pass - - class Get(Operation): - def __init__(self, values: List[str]): - self.values = values - - class GetWithDefault(Operation): - def __init__(self, values: List[Optional[str]]): - self.values = values - - class GetOption(Operation): - def __init__(self, values: List[Optional[str]]): - self.values = values - - class GetAll(Operation): - def __init__(self, pairs: List[Tuple[str, str]]): - self.pairs = pairs - - class Unset(Operation): - pass - - class IsModifiable(Operation): - def __init__(self, bools: List[bool]): - self.bools = bools - - def __init__(self, operation: Operation, warnings: List[str]): - self.operation = operation + def __init__(self, pairs: List[Tuple[str, Optional[str]]], warnings: List[str]): + self.pairs = pairs self.warnings = warnings @classmethod def fromProto(cls, pb: pb2.ConfigResponse) -> "ConfigResult": - warnings = list(pb.warnings) - op_type = pb.operation.WhichOneof("op_type") - if op_type == "set": - return ConfigResult(ConfigResult.Set(), warnings) - elif op_type == "get": - return ConfigResult(ConfigResult.Get(list(pb.operation.get.values)), warnings) - elif op_type == "get_with_default": - return ConfigResult( - ConfigResult.GetWithDefault( - [ - value.value if value.HasField("value") else None - for value in pb.operation.get_with_default.values - ] - ), - warnings, - ) - elif op_type == "get_option": - return ConfigResult( - ConfigResult.GetOption( - [ - value.value if value.HasField("value") else None - for value in pb.operation.get_with_default.values - ] - ), - warnings, - ) - elif op_type == "get_all": - return ConfigResult( - ConfigResult.GetAll( - [(pair.key, pair.value) for pair in pb.operation.get_all.pairs] - ), - warnings, - ) - elif op_type == "unset": - return ConfigResult(ConfigResult.Unset(), warnings) - elif op_type == "is_modifiable": - return ConfigResult( - ConfigResult.IsModifiable(list(pb.operation.is_modifiable.bools)), warnings - ) - else: - raise ValueError(f"Unknown op_type in ConfigResponse: {op_type}") + return ConfigResult( + pairs=[ + (pair.key, pair.value.value if pair.value.HasField("value") else None) + for pair in pb.pairs + ], + warnings=list(pb.warnings), + ) class SparkConnectClient(object): diff --git a/python/pyspark/sql/connect/conf.py b/python/pyspark/sql/connect/conf.py index 2a4ab3fdd4746..23431941f67de 100644 --- a/python/pyspark/sql/connect/conf.py +++ b/python/pyspark/sql/connect/conf.py @@ -19,10 +19,9 @@ from pyspark import _NoValue from pyspark._globals import _NoValueType -from pyspark.errors import IllegalArgumentException from pyspark.sql.conf import RuntimeConfig as PySparkRuntimeConfig from pyspark.sql.connect import proto -from pyspark.sql.connect.client import ConfigResult, SparkConnectClient +from pyspark.sql.connect.client import SparkConnectClient class RuntimeConf: @@ -37,10 +36,9 @@ def set(self, key: str, value: Union[str, int, bool]) -> None: value = "true" if value else "false" elif isinstance(value, int): value = str(value) - op_set = proto.ConfigRequest.Set(pairs=[_key_value(key, value)]) + op_set = proto.ConfigRequest.Set(pairs=[_to_proto_key_value(key, value)]) operation = proto.ConfigRequest.Operation(set=op_set) result = self._client.config(operation) - assert isinstance(result.operation, ConfigResult.Set) for warn in result.warnings: warnings.warn(warn) @@ -53,19 +51,15 @@ def get( if default is _NoValue: op_get = proto.ConfigRequest.Get(keys=[key]) operation = proto.ConfigRequest.Operation(get=op_get) - result = self._client.config(operation) - assert isinstance(result.operation, ConfigResult.Get) - return result.operation.values[0] else: if default is not None: self._checkType(default, "default") op_get_with_default = proto.ConfigRequest.GetWithDefault( - pairs=[_optional_key_value(key, cast(Optional[str], default))] + pairs=[_to_proto_key_value(key, cast(Optional[str], default))] ) operation = proto.ConfigRequest.Operation(get_with_default=op_get_with_default) - result = self._client.config(operation) - assert isinstance(result.operation, ConfigResult.GetWithDefault) - return result.operation.values[0] + result = self._client.config(operation) + return result.pairs[0][1] get.__doc__ = PySparkRuntimeConfig.get.__doc__ @@ -73,7 +67,6 @@ def unset(self, key: str) -> None: op_unset = proto.ConfigRequest.Unset(keys=[key]) operation = proto.ConfigRequest.Operation(unset=op_unset) result = self._client.config(operation) - assert isinstance(result.operation, ConfigResult.Unset) for warn in result.warnings: warnings.warn(warn) @@ -83,8 +76,12 @@ def isModifiable(self, key: str) -> bool: op_is_modifiable = proto.ConfigRequest.IsModifiable(keys=[key]) operation = proto.ConfigRequest.Operation(is_modifiable=op_is_modifiable) result = self._client.config(operation) - assert isinstance(result.operation, ConfigResult.IsModifiable) - return result.operation.bools[0] + if result.pairs[0][1] == "true": + return True + elif result.pairs[0][1] == "false": + return False + else: + raise ValueError(f"Unknown boolean value: {result.pairs[0][1]}") isModifiable.__doc__ = PySparkRuntimeConfig.isModifiable.__doc__ @@ -96,18 +93,12 @@ def _checkType(self, obj: Any, identifier: str) -> None: ) -def _optional_value(value: Optional[str]) -> proto.OptionalValue: +def _to_proto_optional_value(value: Optional[str]) -> proto.OptionalValue: return proto.OptionalValue(value=value) -def _key_value(key: str, value: Optional[str]) -> proto.KeyValue: - if value is None: - raise IllegalArgumentException(f"requirement failed: value cannot be null for key: {key}") - return proto.KeyValue(key=key, value=value) - - -def _optional_key_value(key: str, value: Optional[str]) -> proto.OptionalKeyValue: - return proto.OptionalKeyValue(key=key, value=_optional_value(value)) +def _to_proto_key_value(key: str, value: Optional[str]) -> proto.KeyValue: + return proto.KeyValue(key=key, value=_to_proto_optional_value(value)) RuntimeConf.__doc__ = PySparkRuntimeConfig.__doc__ diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 93bf4e3bd7a80..6db37023854c0 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"2\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value"X\n\x10OptionalKeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\x8a\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1aG\n\x0eGetWithDefault\x12\x35\n\x05pairs\x18\x01 \x03(\x0b\x32\x1f.spark.connect.OptionalKeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB\x0e\n\x0c_client_type"\xa5\x07\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12\x45\n\toperation\x18\x02 \x01(\x0b\x32\'.spark.connect.ConfigResponse.OperationR\toperation\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings\x1a\xf9\x03\n\tOperation\x12\x35\n\x03set\x18\x01 \x01(\x0b\x32!.spark.connect.ConfigResponse.SetH\x00R\x03set\x12\x35\n\x03get\x18\x02 \x01(\x0b\x32!.spark.connect.ConfigResponse.GetH\x00R\x03get\x12X\n\x10get_with_default\x18\x03 \x01(\x0b\x32,.spark.connect.ConfigResponse.GetWithDefaultH\x00R\x0egetWithDefault\x12H\n\nget_option\x18\x04 \x01(\x0b\x32\'.spark.connect.ConfigResponse.GetOptionH\x00R\tgetOption\x12?\n\x07get_all\x18\x05 \x01(\x0b\x32$.spark.connect.ConfigResponse.GetAllH\x00R\x06getAll\x12;\n\x05unset\x18\x06 \x01(\x0b\x32#.spark.connect.ConfigResponse.UnsetH\x00R\x05unset\x12Q\n\ris_modifiable\x18\x07 \x01(\x0b\x32*.spark.connect.ConfigResponse.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x05\n\x03Set\x1a\x1d\n\x03Get\x12\x16\n\x06values\x18\x01 \x03(\tR\x06values\x1a\x46\n\x0eGetWithDefault\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x41\n\tGetOption\x12\x34\n\x06values\x18\x01 \x03(\x0b\x32\x1c.spark.connect.OptionalValueR\x06values\x1a\x37\n\x06GetAll\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x07\n\x05Unset\x1a$\n\x0cIsModifiable\x12\x14\n\x05\x62ools\x18\x01 \x03(\x08R\x05\x62ools2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"P\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\x82\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB\x0e\n\x0c_client_type"x\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -60,7 +60,6 @@ ] _OPTIONALVALUE = DESCRIPTOR.message_types_by_name["OptionalValue"] _KEYVALUE = DESCRIPTOR.message_types_by_name["KeyValue"] -_OPTIONALKEYVALUE = DESCRIPTOR.message_types_by_name["OptionalKeyValue"] _CONFIGREQUEST = DESCRIPTOR.message_types_by_name["ConfigRequest"] _CONFIGREQUEST_OPERATION = _CONFIGREQUEST.nested_types_by_name["Operation"] _CONFIGREQUEST_SET = _CONFIGREQUEST.nested_types_by_name["Set"] @@ -71,14 +70,6 @@ _CONFIGREQUEST_UNSET = _CONFIGREQUEST.nested_types_by_name["Unset"] _CONFIGREQUEST_ISMODIFIABLE = _CONFIGREQUEST.nested_types_by_name["IsModifiable"] _CONFIGRESPONSE = DESCRIPTOR.message_types_by_name["ConfigResponse"] -_CONFIGRESPONSE_OPERATION = _CONFIGRESPONSE.nested_types_by_name["Operation"] -_CONFIGRESPONSE_SET = _CONFIGRESPONSE.nested_types_by_name["Set"] -_CONFIGRESPONSE_GET = _CONFIGRESPONSE.nested_types_by_name["Get"] -_CONFIGRESPONSE_GETWITHDEFAULT = _CONFIGRESPONSE.nested_types_by_name["GetWithDefault"] -_CONFIGRESPONSE_GETOPTION = _CONFIGRESPONSE.nested_types_by_name["GetOption"] -_CONFIGRESPONSE_GETALL = _CONFIGRESPONSE.nested_types_by_name["GetAll"] -_CONFIGRESPONSE_UNSET = _CONFIGRESPONSE.nested_types_by_name["Unset"] -_CONFIGRESPONSE_ISMODIFIABLE = _CONFIGRESPONSE.nested_types_by_name["IsModifiable"] _EXPLAIN_EXPLAINMODE = _EXPLAIN.enum_types_by_name["ExplainMode"] Plan = _reflection.GeneratedProtocolMessageType( "Plan", @@ -229,17 +220,6 @@ ) _sym_db.RegisterMessage(KeyValue) -OptionalKeyValue = _reflection.GeneratedProtocolMessageType( - "OptionalKeyValue", - (_message.Message,), - { - "DESCRIPTOR": _OPTIONALKEYVALUE, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.OptionalKeyValue) - }, -) -_sym_db.RegisterMessage(OptionalKeyValue) - ConfigRequest = _reflection.GeneratedProtocolMessageType( "ConfigRequest", (_message.Message,), @@ -335,92 +315,12 @@ "ConfigResponse", (_message.Message,), { - "Operation": _reflection.GeneratedProtocolMessageType( - "Operation", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_OPERATION, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Operation) - }, - ), - "Set": _reflection.GeneratedProtocolMessageType( - "Set", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_SET, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Set) - }, - ), - "Get": _reflection.GeneratedProtocolMessageType( - "Get", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_GET, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Get) - }, - ), - "GetWithDefault": _reflection.GeneratedProtocolMessageType( - "GetWithDefault", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_GETWITHDEFAULT, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.GetWithDefault) - }, - ), - "GetOption": _reflection.GeneratedProtocolMessageType( - "GetOption", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_GETOPTION, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.GetOption) - }, - ), - "GetAll": _reflection.GeneratedProtocolMessageType( - "GetAll", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_GETALL, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.GetAll) - }, - ), - "Unset": _reflection.GeneratedProtocolMessageType( - "Unset", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_UNSET, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.Unset) - }, - ), - "IsModifiable": _reflection.GeneratedProtocolMessageType( - "IsModifiable", - (_message.Message,), - { - "DESCRIPTOR": _CONFIGRESPONSE_ISMODIFIABLE, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse.IsModifiable) - }, - ), "DESCRIPTOR": _CONFIGRESPONSE, "__module__": "spark.connect.base_pb2" # @@protoc_insertion_point(class_scope:spark.connect.ConfigResponse) }, ) _sym_db.RegisterMessage(ConfigResponse) -_sym_db.RegisterMessage(ConfigResponse.Operation) -_sym_db.RegisterMessage(ConfigResponse.Set) -_sym_db.RegisterMessage(ConfigResponse.Get) -_sym_db.RegisterMessage(ConfigResponse.GetWithDefault) -_sym_db.RegisterMessage(ConfigResponse.GetOption) -_sym_db.RegisterMessage(ConfigResponse.GetAll) -_sym_db.RegisterMessage(ConfigResponse.Unset) -_sym_db.RegisterMessage(ConfigResponse.IsModifiable) _SPARKCONNECTSERVICE = DESCRIPTOR.services_by_name["SparkConnectService"] if _descriptor._USE_C_DESCRIPTORS == False: @@ -458,45 +358,27 @@ _OPTIONALVALUE._serialized_start = 2109 _OPTIONALVALUE._serialized_end = 2161 _KEYVALUE._serialized_start = 2163 - _KEYVALUE._serialized_end = 2213 - _OPTIONALKEYVALUE._serialized_start = 2215 - _OPTIONALKEYVALUE._serialized_end = 2303 - _CONFIGREQUEST._serialized_start = 2306 - _CONFIGREQUEST._serialized_end = 3340 - _CONFIGREQUEST_OPERATION._serialized_start = 2524 - _CONFIGREQUEST_OPERATION._serialized_end = 3022 - _CONFIGREQUEST_SET._serialized_start = 3024 - _CONFIGREQUEST_SET._serialized_end = 3076 - _CONFIGREQUEST_GET._serialized_start = 3078 - _CONFIGREQUEST_GET._serialized_end = 3103 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 3105 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3176 - _CONFIGREQUEST_GETOPTION._serialized_start = 3178 - _CONFIGREQUEST_GETOPTION._serialized_end = 3209 - _CONFIGREQUEST_GETALL._serialized_start = 3211 - _CONFIGREQUEST_GETALL._serialized_end = 3259 - _CONFIGREQUEST_UNSET._serialized_start = 3261 - _CONFIGREQUEST_UNSET._serialized_end = 3288 - _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3290 - _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3324 - _CONFIGRESPONSE._serialized_start = 3343 - _CONFIGRESPONSE._serialized_end = 4276 - _CONFIGRESPONSE_OPERATION._serialized_start = 3490 - _CONFIGRESPONSE_OPERATION._serialized_end = 3995 - _CONFIGRESPONSE_SET._serialized_start = 3024 - _CONFIGRESPONSE_SET._serialized_end = 3029 - _CONFIGRESPONSE_GET._serialized_start = 4004 - _CONFIGRESPONSE_GET._serialized_end = 4033 - _CONFIGRESPONSE_GETWITHDEFAULT._serialized_start = 4035 - _CONFIGRESPONSE_GETWITHDEFAULT._serialized_end = 4105 - _CONFIGRESPONSE_GETOPTION._serialized_start = 4107 - _CONFIGRESPONSE_GETOPTION._serialized_end = 4172 - _CONFIGRESPONSE_GETALL._serialized_start = 4174 - _CONFIGRESPONSE_GETALL._serialized_end = 4229 - _CONFIGRESPONSE_UNSET._serialized_start = 3261 - _CONFIGRESPONSE_UNSET._serialized_end = 3268 - _CONFIGRESPONSE_ISMODIFIABLE._serialized_start = 4240 - _CONFIGRESPONSE_ISMODIFIABLE._serialized_end = 4276 - _SPARKCONNECTSERVICE._serialized_start = 4279 - _SPARKCONNECTSERVICE._serialized_end = 4551 + _KEYVALUE._serialized_end = 2243 + _CONFIGREQUEST._serialized_start = 2246 + _CONFIGREQUEST._serialized_end = 3272 + _CONFIGREQUEST_OPERATION._serialized_start = 2464 + _CONFIGREQUEST_OPERATION._serialized_end = 2962 + _CONFIGREQUEST_SET._serialized_start = 2964 + _CONFIGREQUEST_SET._serialized_end = 3016 + _CONFIGREQUEST_GET._serialized_start = 3018 + _CONFIGREQUEST_GET._serialized_end = 3043 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 3045 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3108 + _CONFIGREQUEST_GETOPTION._serialized_start = 3110 + _CONFIGREQUEST_GETOPTION._serialized_end = 3141 + _CONFIGREQUEST_GETALL._serialized_start = 3143 + _CONFIGREQUEST_GETALL._serialized_end = 3191 + _CONFIGREQUEST_UNSET._serialized_start = 3193 + _CONFIGREQUEST_UNSET._serialized_end = 3220 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3222 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3256 + _CONFIGRESPONSE._serialized_start = 3274 + _CONFIGRESPONSE._serialized_end = 3394 + _SPARKCONNECTSERVICE._serialized_start = 3397 + _SPARKCONNECTSERVICE._serialized_end = 3669 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index c5ae189add4fa..b6abd64f82858 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -600,27 +600,6 @@ class KeyValue(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - KEY_FIELD_NUMBER: builtins.int - VALUE_FIELD_NUMBER: builtins.int - key: builtins.str - value: builtins.str - def __init__( - self, - *, - key: builtins.str = ..., - value: builtins.str = ..., - ) -> None: ... - def ClearField( - self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] - ) -> None: ... - -global___KeyValue = KeyValue - -class OptionalKeyValue(google.protobuf.message.Message): - """The key-value pair for the config request and response when the value can be optional.""" - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - KEY_FIELD_NUMBER: builtins.int VALUE_FIELD_NUMBER: builtins.int key: builtins.str @@ -639,7 +618,7 @@ class OptionalKeyValue(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... -global___OptionalKeyValue = OptionalKeyValue +global___KeyValue = KeyValue class ConfigRequest(google.protobuf.message.Message): """Request to update or fetch the configurations.""" @@ -768,14 +747,12 @@ class ConfigRequest(google.protobuf.message.Message): @property def pairs( self, - ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ - global___OptionalKeyValue - ]: + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[global___KeyValue]: """(Required) The config key-value paris to get. The value will be used as the default value.""" def __init__( self, *, - pairs: collections.abc.Iterable[global___OptionalKeyValue] | None = ..., + pairs: collections.abc.Iterable[global___KeyValue] | None = ..., ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... @@ -916,197 +893,19 @@ class ConfigResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - class Operation(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - SET_FIELD_NUMBER: builtins.int - GET_FIELD_NUMBER: builtins.int - GET_WITH_DEFAULT_FIELD_NUMBER: builtins.int - GET_OPTION_FIELD_NUMBER: builtins.int - GET_ALL_FIELD_NUMBER: builtins.int - UNSET_FIELD_NUMBER: builtins.int - IS_MODIFIABLE_FIELD_NUMBER: builtins.int - @property - def set(self) -> global___ConfigResponse.Set: ... - @property - def get(self) -> global___ConfigResponse.Get: ... - @property - def get_with_default(self) -> global___ConfigResponse.GetWithDefault: ... - @property - def get_option(self) -> global___ConfigResponse.GetOption: ... - @property - def get_all(self) -> global___ConfigResponse.GetAll: ... - @property - def unset(self) -> global___ConfigResponse.Unset: ... - @property - def is_modifiable(self) -> global___ConfigResponse.IsModifiable: ... - def __init__( - self, - *, - set: global___ConfigResponse.Set | None = ..., - get: global___ConfigResponse.Get | None = ..., - get_with_default: global___ConfigResponse.GetWithDefault | None = ..., - get_option: global___ConfigResponse.GetOption | None = ..., - get_all: global___ConfigResponse.GetAll | None = ..., - unset: global___ConfigResponse.Unset | None = ..., - is_modifiable: global___ConfigResponse.IsModifiable | None = ..., - ) -> None: ... - def HasField( - self, - field_name: typing_extensions.Literal[ - "get", - b"get", - "get_all", - b"get_all", - "get_option", - b"get_option", - "get_with_default", - b"get_with_default", - "is_modifiable", - b"is_modifiable", - "op_type", - b"op_type", - "set", - b"set", - "unset", - b"unset", - ], - ) -> builtins.bool: ... - def ClearField( - self, - field_name: typing_extensions.Literal[ - "get", - b"get", - "get_all", - b"get_all", - "get_option", - b"get_option", - "get_with_default", - b"get_with_default", - "is_modifiable", - b"is_modifiable", - "op_type", - b"op_type", - "set", - b"set", - "unset", - b"unset", - ], - ) -> None: ... - def WhichOneof( - self, oneof_group: typing_extensions.Literal["op_type", b"op_type"] - ) -> typing_extensions.Literal[ - "set", "get", "get_with_default", "get_option", "get_all", "unset", "is_modifiable" - ] | None: ... - - class Set(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - def __init__( - self, - ) -> None: ... - - class Get(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - VALUES_FIELD_NUMBER: builtins.int - @property - def values( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: ... - def __init__( - self, - *, - values: collections.abc.Iterable[builtins.str] | None = ..., - ) -> None: ... - def ClearField( - self, field_name: typing_extensions.Literal["values", b"values"] - ) -> None: ... - - class GetWithDefault(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - VALUES_FIELD_NUMBER: builtins.int - @property - def values( - self, - ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ - global___OptionalValue - ]: ... - def __init__( - self, - *, - values: collections.abc.Iterable[global___OptionalValue] | None = ..., - ) -> None: ... - def ClearField( - self, field_name: typing_extensions.Literal["values", b"values"] - ) -> None: ... - - class GetOption(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - VALUES_FIELD_NUMBER: builtins.int - @property - def values( - self, - ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ - global___OptionalValue - ]: ... - def __init__( - self, - *, - values: collections.abc.Iterable[global___OptionalValue] | None = ..., - ) -> None: ... - def ClearField( - self, field_name: typing_extensions.Literal["values", b"values"] - ) -> None: ... - - class GetAll(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - PAIRS_FIELD_NUMBER: builtins.int - @property - def pairs( - self, - ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ - global___KeyValue - ]: ... - def __init__( - self, - *, - pairs: collections.abc.Iterable[global___KeyValue] | None = ..., - ) -> None: ... - def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... - - class Unset(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - def __init__( - self, - ) -> None: ... - - class IsModifiable(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - BOOLS_FIELD_NUMBER: builtins.int - @property - def bools( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.bool]: ... - def __init__( - self, - *, - bools: collections.abc.Iterable[builtins.bool] | None = ..., - ) -> None: ... - def ClearField(self, field_name: typing_extensions.Literal["bools", b"bools"]) -> None: ... - CLIENT_ID_FIELD_NUMBER: builtins.int - OPERATION_FIELD_NUMBER: builtins.int + PAIRS_FIELD_NUMBER: builtins.int WARNINGS_FIELD_NUMBER: builtins.int client_id: builtins.str @property - def operation(self) -> global___ConfigResponse.Operation: - """(Required) The operation for the config.""" + def pairs( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[global___KeyValue]: + """(Optional) The result key-value pairs. + + Available when the operation is 'Get', 'GetWithDefault', 'GetOption', 'GetAll'. + Also available for the operation 'IsModifiable' with boolean string "true" and "false". + """ @property def warnings( self, @@ -1119,16 +918,13 @@ class ConfigResponse(google.protobuf.message.Message): self, *, client_id: builtins.str = ..., - operation: global___ConfigResponse.Operation | None = ..., + pairs: collections.abc.Iterable[global___KeyValue] | None = ..., warnings: collections.abc.Iterable[builtins.str] | None = ..., ) -> None: ... - def HasField( - self, field_name: typing_extensions.Literal["operation", b"operation"] - ) -> builtins.bool: ... def ClearField( self, field_name: typing_extensions.Literal[ - "client_id", b"client_id", "operation", b"operation", "warnings", b"warnings" + "client_id", b"client_id", "pairs", b"pairs", "warnings", b"warnings" ], ) -> None: ... diff --git a/python/pyspark/sql/tests/test_conf.py b/python/pyspark/sql/tests/test_conf.py index 499db606ed9d8..15722c2c57a40 100644 --- a/python/pyspark/sql/tests/test_conf.py +++ b/python/pyspark/sql/tests/test_conf.py @@ -44,6 +44,9 @@ def test_conf(self): # `defaultValue` in `spark.conf.get` is set to None. self.assertEqual(spark.conf.get("spark.sql.sources.partitionOverwriteMode", None), None) + self.assertTrue(spark.conf.isModifiable("spark.sql.execution.arrow.maxRecordsPerBatch")) + self.assertFalse(spark.conf.isModifiable("spark.sql.warehouse.dir")) + def test_conf_with_python_objects(self): spark = self.spark @@ -60,6 +63,8 @@ def test_conf_with_python_objects(self): with self.assertRaises(Exception): spark.conf.set("foo", Decimal(1)) + spark.conf.unset("foo") + class ConfTests(ConfTestsMixin, ReusedSQLTestCase): pass From 61034d269281029885cb333a1756b0d16fef6387 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 24 Feb 2023 12:00:29 -0800 Subject: [PATCH 14/14] Fix. --- .../main/protobuf/spark/connect/base.proto | 9 +-- .../service/SparkConnectConfigHandler.scala | 26 +++----- python/pyspark/sql/connect/client.py | 5 +- python/pyspark/sql/connect/conf.py | 12 +--- python/pyspark/sql/connect/proto/base_pb2.py | 64 ++++++++----------- python/pyspark/sql/connect/proto/base_pb2.pyi | 39 ++++------- 6 files changed, 52 insertions(+), 103 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 4cd6dca23a43d..1ffbb8aa8814f 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -183,15 +183,12 @@ message ExecutePlanResponse { } } -// The placeholder for the config request and response when the values can be optional. -message OptionalValue { - optional string value = 1; -} - // The key-value pair for the config request and response. message KeyValue { + // (Required) The key. string key = 1; - OptionalValue value = 2; + // (Optional) The value. + optional string value = 2; } // Request to update or fetch the configurations. diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index 92a506934e220..84f625222a856 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -63,9 +63,8 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes conf: RuntimeConfig): proto.ConfigResponse.Builder = { val builder = proto.ConfigResponse.newBuilder() operation.getPairsList.asScala.iterator.foreach { pair => - val key = pair.getKey - val value = SparkConnectConfigHandler.toOption(pair.getValue).orNull - conf.set(key, value) + val (key, value) = SparkConnectConfigHandler.toKeyValue(pair) + conf.set(key, value.orNull) getWarning(key).foreach(builder.addWarnings) } builder @@ -88,9 +87,8 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes conf: RuntimeConfig): proto.ConfigResponse.Builder = { val builder = proto.ConfigResponse.newBuilder() operation.getPairsList.asScala.iterator.foreach { pair => - val key = pair.getKey - val default = SparkConnectConfigHandler.toOption(pair.getValue).orNull - val value = conf.get(key, default) + val (key, default) = SparkConnectConfigHandler.toKeyValue(pair) + val value = conf.get(key, default.orNull) builder.addPairs(SparkConnectConfigHandler.toProtoKeyValue(key, Option(value))) getWarning(key).foreach(builder.addWarnings) } @@ -164,24 +162,20 @@ object SparkConnectConfigHandler { private[connect] val unsupportedConfigurations = Set("spark.sql.execution.arrow.enabled") - def toOption(value: proto.OptionalValue): Option[String] = { - if (value.hasValue) { - Some(value.getValue) + def toKeyValue(pair: proto.KeyValue): (String, Option[String]) = { + val key = pair.getKey + val value = if (pair.hasValue) { + Some(pair.getValue) } else { None } - } - - def toProtoOptionalValue(value: Option[String]): proto.OptionalValue = { - val builder = proto.OptionalValue.newBuilder() - value.foreach(builder.setValue) - builder.build() + (key, value) } def toProtoKeyValue(key: String, value: Option[String]): proto.KeyValue = { val builder = proto.KeyValue.newBuilder() builder.setKey(key) - builder.setValue(toProtoOptionalValue(value)) + value.foreach(builder.setValue) builder.build() } } diff --git a/python/pyspark/sql/connect/client.py b/python/pyspark/sql/connect/client.py index 128020953a759..6ec10897fa40e 100644 --- a/python/pyspark/sql/connect/client.py +++ b/python/pyspark/sql/connect/client.py @@ -410,10 +410,7 @@ def __init__(self, pairs: List[Tuple[str, Optional[str]]], warnings: List[str]): @classmethod def fromProto(cls, pb: pb2.ConfigResponse) -> "ConfigResult": return ConfigResult( - pairs=[ - (pair.key, pair.value.value if pair.value.HasField("value") else None) - for pair in pb.pairs - ], + pairs=[(pair.key, pair.value if pair.HasField("value") else None) for pair in pb.pairs], warnings=list(pb.warnings), ) diff --git a/python/pyspark/sql/connect/conf.py b/python/pyspark/sql/connect/conf.py index 23431941f67de..d323de716c46a 100644 --- a/python/pyspark/sql/connect/conf.py +++ b/python/pyspark/sql/connect/conf.py @@ -36,7 +36,7 @@ def set(self, key: str, value: Union[str, int, bool]) -> None: value = "true" if value else "false" elif isinstance(value, int): value = str(value) - op_set = proto.ConfigRequest.Set(pairs=[_to_proto_key_value(key, value)]) + op_set = proto.ConfigRequest.Set(pairs=[proto.KeyValue(key=key, value=value)]) operation = proto.ConfigRequest.Operation(set=op_set) result = self._client.config(operation) for warn in result.warnings: @@ -55,7 +55,7 @@ def get( if default is not None: self._checkType(default, "default") op_get_with_default = proto.ConfigRequest.GetWithDefault( - pairs=[_to_proto_key_value(key, cast(Optional[str], default))] + pairs=[proto.KeyValue(key=key, value=cast(Optional[str], default))] ) operation = proto.ConfigRequest.Operation(get_with_default=op_get_with_default) result = self._client.config(operation) @@ -93,14 +93,6 @@ def _checkType(self, obj: Any, identifier: str) -> None: ) -def _to_proto_optional_value(value: Optional[str]) -> proto.OptionalValue: - return proto.OptionalValue(value=value) - - -def _to_proto_key_value(key: str, value: Optional[str]) -> proto.KeyValue: - return proto.KeyValue(key=key, value=_to_proto_optional_value(value)) - - RuntimeConf.__doc__ = PySparkRuntimeConfig.__doc__ diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 6db37023854c0..95951d8f8e3b6 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"4\n\rOptionalValue\x12\x19\n\x05value\x18\x01 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"P\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x32\n\x05value\x18\x02 \x01(\x0b\x32\x1c.spark.connect.OptionalValueR\x05value"\x82\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB\x0e\n\x0c_client_type"x\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x02\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString\x12\x1f\n\x0btree_string\x18\x04 \x01(\tR\ntreeString\x12\x19\n\x08is_local\x18\x05 \x01(\x08R\x07isLocal\x12!\n\x0cis_streaming\x18\x06 \x01(\x08R\x0bisStreaming\x12\x1f\n\x0binput_files\x18\x07 \x03(\tR\ninputFiles"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\x82\x08\n\rConfigRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB\x0e\n\x0c_client_type"x\n\x0e\x43onfigResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings2\x90\x02\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -58,7 +58,6 @@ _EXECUTEPLANRESPONSE_METRICS_METRICVALUE = _EXECUTEPLANRESPONSE_METRICS.nested_types_by_name[ "MetricValue" ] -_OPTIONALVALUE = DESCRIPTOR.message_types_by_name["OptionalValue"] _KEYVALUE = DESCRIPTOR.message_types_by_name["KeyValue"] _CONFIGREQUEST = DESCRIPTOR.message_types_by_name["ConfigRequest"] _CONFIGREQUEST_OPERATION = _CONFIGREQUEST.nested_types_by_name["Operation"] @@ -198,17 +197,6 @@ _sym_db.RegisterMessage(ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntry) _sym_db.RegisterMessage(ExecutePlanResponse.Metrics.MetricValue) -OptionalValue = _reflection.GeneratedProtocolMessageType( - "OptionalValue", - (_message.Message,), - { - "DESCRIPTOR": _OPTIONALVALUE, - "__module__": "spark.connect.base_pb2" - # @@protoc_insertion_point(class_scope:spark.connect.OptionalValue) - }, -) -_sym_db.RegisterMessage(OptionalValue) - KeyValue = _reflection.GeneratedProtocolMessageType( "KeyValue", (_message.Message,), @@ -355,30 +343,28 @@ _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 2017 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 2019 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 2107 - _OPTIONALVALUE._serialized_start = 2109 - _OPTIONALVALUE._serialized_end = 2161 - _KEYVALUE._serialized_start = 2163 - _KEYVALUE._serialized_end = 2243 - _CONFIGREQUEST._serialized_start = 2246 - _CONFIGREQUEST._serialized_end = 3272 - _CONFIGREQUEST_OPERATION._serialized_start = 2464 - _CONFIGREQUEST_OPERATION._serialized_end = 2962 - _CONFIGREQUEST_SET._serialized_start = 2964 - _CONFIGREQUEST_SET._serialized_end = 3016 - _CONFIGREQUEST_GET._serialized_start = 3018 - _CONFIGREQUEST_GET._serialized_end = 3043 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 3045 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3108 - _CONFIGREQUEST_GETOPTION._serialized_start = 3110 - _CONFIGREQUEST_GETOPTION._serialized_end = 3141 - _CONFIGREQUEST_GETALL._serialized_start = 3143 - _CONFIGREQUEST_GETALL._serialized_end = 3191 - _CONFIGREQUEST_UNSET._serialized_start = 3193 - _CONFIGREQUEST_UNSET._serialized_end = 3220 - _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3222 - _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3256 - _CONFIGRESPONSE._serialized_start = 3274 - _CONFIGRESPONSE._serialized_end = 3394 - _SPARKCONNECTSERVICE._serialized_start = 3397 - _SPARKCONNECTSERVICE._serialized_end = 3669 + _KEYVALUE._serialized_start = 2109 + _KEYVALUE._serialized_end = 2174 + _CONFIGREQUEST._serialized_start = 2177 + _CONFIGREQUEST._serialized_end = 3203 + _CONFIGREQUEST_OPERATION._serialized_start = 2395 + _CONFIGREQUEST_OPERATION._serialized_end = 2893 + _CONFIGREQUEST_SET._serialized_start = 2895 + _CONFIGREQUEST_SET._serialized_end = 2947 + _CONFIGREQUEST_GET._serialized_start = 2949 + _CONFIGREQUEST_GET._serialized_end = 2974 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 2976 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 3039 + _CONFIGREQUEST_GETOPTION._serialized_start = 3041 + _CONFIGREQUEST_GETOPTION._serialized_end = 3072 + _CONFIGREQUEST_GETALL._serialized_start = 3074 + _CONFIGREQUEST_GETALL._serialized_end = 3122 + _CONFIGREQUEST_UNSET._serialized_start = 3124 + _CONFIGREQUEST_UNSET._serialized_end = 3151 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 3153 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 3187 + _CONFIGRESPONSE._serialized_start = 3205 + _CONFIGRESPONSE._serialized_end = 3325 + _SPARKCONNECTSERVICE._serialized_start = 3328 + _SPARKCONNECTSERVICE._serialized_end = 3600 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index b6abd64f82858..f6c402b229f9e 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -571,53 +571,36 @@ class ExecutePlanResponse(google.protobuf.message.Message): global___ExecutePlanResponse = ExecutePlanResponse -class OptionalValue(google.protobuf.message.Message): - """The placeholder for the config request and response when the values can be optional.""" +class KeyValue(google.protobuf.message.Message): + """The key-value pair for the config request and response.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + KEY_FIELD_NUMBER: builtins.int VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + """(Required) The key.""" value: builtins.str + """(Optional) The value.""" def __init__( self, *, + key: builtins.str = ..., value: builtins.str | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal["_value", b"_value", "value", b"value"] ) -> builtins.bool: ... def ClearField( - self, field_name: typing_extensions.Literal["_value", b"_value", "value", b"value"] + self, + field_name: typing_extensions.Literal[ + "_value", b"_value", "key", b"key", "value", b"value" + ], ) -> None: ... def WhichOneof( self, oneof_group: typing_extensions.Literal["_value", b"_value"] ) -> typing_extensions.Literal["value"] | None: ... -global___OptionalValue = OptionalValue - -class KeyValue(google.protobuf.message.Message): - """The key-value pair for the config request and response.""" - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - KEY_FIELD_NUMBER: builtins.int - VALUE_FIELD_NUMBER: builtins.int - key: builtins.str - @property - def value(self) -> global___OptionalValue: ... - def __init__( - self, - *, - key: builtins.str = ..., - value: global___OptionalValue | None = ..., - ) -> None: ... - def HasField( - self, field_name: typing_extensions.Literal["value", b"value"] - ) -> builtins.bool: ... - def ClearField( - self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] - ) -> None: ... - global___KeyValue = KeyValue class ConfigRequest(google.protobuf.message.Message):