From fb38900b2a06a2a486826f4a44305da30ad7e474 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Thu, 21 Mar 2024 14:59:50 +0100 Subject: [PATCH 01/20] wip --- .../scala/org/apache/spark/sql/Dataset.scala | 16 +++++++++++++--- .../org/apache/spark/sql/SparkSession.scala | 9 ++++++--- .../apache/spark/sql/ClientE2ETestSuite.scala | 5 +++++ 3 files changed, 24 insertions(+), 6 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 9a42afebf8f2..f3a97b22e769 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -28,6 +28,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.function._ import org.apache.spark.connect.proto +import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ @@ -3337,9 +3338,14 @@ class Dataset[T] private[sql] ( } } - def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = { - throw new UnsupportedOperationException("observe is not implemented.") - } + def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = + sparkSession.newDataset(agnosticEncoder) { + builder => + builder.getCollectMetricsBuilder + .setInput(plan.getRoot) + .setName(name) + .addAllMetrics((expr +: exprs).map(_.expr).asJava) + } def checkpoint(): Dataset[T] = { throw new UnsupportedOperationException("checkpoint is not implemented.") @@ -3399,6 +3405,10 @@ class Dataset[T] private[sql] ( def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder) + def collectObservation(): Seq[ObservedMetrics] = { + sparkSession.execute(plan).flatMap(r => r.getObservedMetricsList().asScala).toSeq + } + private[sql] def withResult[E](f: SparkResult[T] => E): E = { val result = collectResult() try f(result) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index adee5b33fb4e..e577ef2fc59d 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalog.Catalog import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{BoxedLongEncoder, UnboundRowEncoder} -import org.apache.spark.sql.connect.client.{ClassFinder, SparkConnectClient, SparkResult} +import org.apache.spark.sql.connect.client.{ClassFinder, CloseableIterator, SparkConnectClient, SparkResult} import org.apache.spark.sql.connect.client.SparkConnectClient.Configuration import org.apache.spark.sql.connect.client.arrow.ArrowSerializer import org.apache.spark.sql.functions.lit @@ -550,15 +550,18 @@ class SparkSession private[sql] ( builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) val plan = proto.Plan.newBuilder().setRoot(builder).build() // .foreach forces that the iterator is consumed and closed - client.execute(plan).foreach(_ => ()) + execute(plan).foreach(_ => ()) } private[sql] def execute(command: proto.Command): Seq[ExecutePlanResponse] = { val plan = proto.Plan.newBuilder().setCommand(command).build() // .toSeq forces that the iterator is consumed and closed - client.execute(plan).toSeq + execute(plan).toSeq } + private[sql] def execute(plan: proto.Plan): CloseableIterator[ExecutePlanResponse] = + client.execute(plan) + private[sql] def registerUdf(udf: proto.CommonInlineUserDefinedFunction): Unit = { val command = proto.Command.newBuilder().setRegisterFunction(udf).build() execute(command) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index f2f1571452c0..62fb316cf142 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -321,6 +321,11 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM .load() val outputFolderPath = Files.createTempDirectory("output").toAbsolutePath + val dd = + df.observe("observation", min("age"), max("age"), sum("age")) + + val ob = dd.collectObservation() + df.write .format("csv") .mode("overwrite") From e534f8269d9ac49bca9034a94c67c36d0909b5aa Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Mon, 25 Mar 2024 15:03:19 +0100 Subject: [PATCH 02/20] add tests --- .../scala/org/apache/spark/sql/Dataset.scala | 131 ++++++++++++------ .../org/apache/spark/sql/Observation.scala | 24 ++++ .../org/apache/spark/sql/SparkSession.scala | 14 +- .../apache/spark/sql/ClientE2ETestSuite.scala | 42 +++++- .../sql/connect/client/SparkResult.scala | 26 +++- .../apache/spark/sql/ObservationBase.scala | 113 +++++++++++++++ .../org/apache/spark/sql/Observation.scala | 62 +-------- 7 files changed, 300 insertions(+), 112 deletions(-) create mode 100644 connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/ObservationBase.scala diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index f3a97b22e769..bd64b4e4a325 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -28,7 +28,6 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.function._ import org.apache.spark.connect.proto -import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ @@ -132,13 +131,24 @@ import org.apache.spark.util.SparkClassUtils class Dataset[T] private[sql] ( val sparkSession: SparkSession, @DeveloperApi val plan: proto.Plan, - val encoder: Encoder[T]) + val encoder: Encoder[T], + @DeveloperApi carryOverObservations: Option[Map[String, Observation]] = None) extends Serializable { // Make sure we don't forget to set plan id. assert(plan.getRoot.getCommon.hasPlanId) private[sql] val agnosticEncoder: AgnosticEncoder[T] = encoderFor(encoder) + private var observationsOpt: Option[mutable.Map[String, Observation]] = { + carryOverObservations match { + case Some(observations) => + Some(mutable.Map.newBuilder[String, Observation].addAll(observations).result()) + case None => None + } + } + + private def getObservationsMapOpt: Option[Map[String, Observation]] = observationsOpt.map(_.toMap) + override def toString: String = { try { val builder = new mutable.StringBuilder @@ -537,7 +547,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def show(numRows: Int, truncate: Int, vertical: Boolean): Unit = { - val df = sparkSession.newDataset(StringEncoder) { builder => + val df = sparkSession.newDataset(StringEncoder, getObservationsMapOpt) { builder => builder.getShowStringBuilder .setInput(plan.getRoot) .setNumRows(numRows) @@ -845,7 +855,7 @@ class Dataset[T] private[sql] ( } private def buildSort(global: Boolean, sortExprs: Seq[Column]): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getSortBuilder .setInput(plan.getRoot) .setIsGlobal(global) @@ -899,7 +909,7 @@ class Dataset[T] private[sql] ( EncoderField(s"_2", other.agnosticEncoder, rightNullable, Metadata.empty)), None) - sparkSession.newDataset(tupleEncoder) { builder => + sparkSession.newDataset(tupleEncoder, getObservationsMapOpt) { builder => val joinBuilder = builder.getJoinBuilder joinBuilder .setLeft(plan.getRoot) @@ -1029,7 +1039,7 @@ class Dataset[T] private[sql] ( */ @scala.annotation.varargs def hint(name: String, parameters: Any*): Dataset[T] = - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getHintBuilder .setInput(plan.getRoot) .setName(name) @@ -1090,10 +1100,12 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def as(alias: String): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => - builder.getSubqueryAliasBuilder - .setInput(plan.getRoot) - .setAlias(alias) + def as(alias: String): Dataset[T] = { + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + builder.getSubqueryAliasBuilder + .setInput(plan.getRoot) + .setAlias(alias) + } } /** @@ -1184,7 +1196,7 @@ class Dataset[T] private[sql] ( } else { c1.expr } - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(encoder, getObservationsMapOpt) { builder => builder.getProjectBuilder .setInput(plan.getRoot) .addExpressions(expr) @@ -1206,7 +1218,7 @@ class Dataset[T] private[sql] ( * methods and typed select methods is the encoder used to build the return dataset. */ private def selectUntyped(encoder: AgnosticEncoder[_], cols: Seq[Column]): Dataset[_] = { - sparkSession.newDataset(encoder) { builder => + sparkSession.newDataset(encoder, getObservationsMapOpt) { builder => builder.getProjectBuilder .setInput(plan.getRoot) .addAllExpressions(cols.map(_.expr).asJava) @@ -1272,10 +1284,11 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def filter(condition: Column): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { - builder => - builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) - } + def filter(condition: Column): Dataset[T] = + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { + builder => + builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) + } /** * Filters rows using the given SQL expression. @@ -1395,7 +1408,7 @@ class Dataset[T] private[sql] ( val reduceExpr = Column.fn("reduce", udf.apply(col("*"), col("*"))).expr val result = sparkSession - .newDataset(agnosticEncoder) { builder => + .newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getAggregateBuilder .setInput(plan.getRoot) .addAggregateExpressions(reduceExpr) @@ -1788,10 +1801,12 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def limit(n: Int): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => - builder.getLimitBuilder - .setInput(plan.getRoot) - .setLimit(n) + def limit(n: Int): Dataset[T] = { + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + builder.getLimitBuilder + .setInput(plan.getRoot) + .setLimit(n) + } } /** @@ -1800,16 +1815,18 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def offset(n: Int): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => - builder.getOffsetBuilder - .setInput(plan.getRoot) - .setOffset(n) + def offset(n: Int): Dataset[T] = { + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + builder.getOffsetBuilder + .setInput(plan.getRoot) + .setOffset(n) + } } private def buildSetOp(right: Dataset[T], setOpType: proto.SetOperation.SetOpType)( f: proto.SetOperation.Builder => Unit): Dataset[T] = { checkSameSparkSession(right) - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => f( builder.getSetOpBuilder .setSetOpType(setOpType) @@ -2082,7 +2099,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getSampleBuilder .setInput(plan.getRoot) .setWithReplacement(withReplacement) @@ -2150,7 +2167,7 @@ class Dataset[T] private[sql] ( normalizedCumWeights .sliding(2) .map { case Array(low, high) => - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getSampleBuilder .setInput(sortedInput) .setWithReplacement(false) @@ -2477,8 +2494,8 @@ class Dataset[T] private[sql] ( private def buildDropDuplicates( columns: Option[Seq[String]], - withinWaterMark: Boolean): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { - builder => + withinWaterMark: Boolean): Dataset[T] = { + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => val dropBuilder = builder.getDeduplicateBuilder .setInput(plan.getRoot) .setWithinWatermark(withinWaterMark) @@ -2487,6 +2504,7 @@ class Dataset[T] private[sql] ( } else { dropBuilder.setAllColumnsAsKeys(true) } + } } /** @@ -2710,7 +2728,7 @@ class Dataset[T] private[sql] ( function = func, inputEncoders = agnosticEncoder :: Nil, outputEncoder = PrimitiveBooleanEncoder) - sparkSession.newDataset[T](agnosticEncoder) { builder => + sparkSession.newDataset[T](agnosticEncoder, getObservationsMapOpt) { builder => builder.getFilterBuilder .setInput(plan.getRoot) .setCondition(udf.apply(col("*")).expr) @@ -2763,7 +2781,7 @@ class Dataset[T] private[sql] ( function = func, inputEncoders = agnosticEncoder :: Nil, outputEncoder = outputEncoder) - sparkSession.newDataset(outputEncoder) { builder => + sparkSession.newDataset(outputEncoder, getObservationsMapOpt) { builder => builder.getMapPartitionsBuilder .setInput(plan.getRoot) .setFunc(udf.apply(col("*")).expr.getCommonInlineUserDefinedFunction) @@ -2931,7 +2949,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def tail(n: Int): Array[T] = { - val lastN = sparkSession.newDataset(agnosticEncoder) { builder => + val lastN = sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getTailBuilder .setInput(plan.getRoot) .setLimit(n) @@ -3002,7 +3020,7 @@ class Dataset[T] private[sql] ( } private def buildRepartition(numPartitions: Int, shuffle: Boolean): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getRepartitionBuilder .setInput(plan.getRoot) .setNumPartitions(numPartitions) @@ -3012,12 +3030,13 @@ class Dataset[T] private[sql] ( private def buildRepartitionByExpression( numPartitions: Option[Int], - partitionExprs: Seq[Column]): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { - builder => + partitionExprs: Seq[Column]): Dataset[T] = { + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => val repartitionBuilder = builder.getRepartitionByExpressionBuilder .setInput(plan.getRoot) .addAllPartitionExprs(partitionExprs.map(_.expr).asJava) numPartitions.foreach(repartitionBuilder.setNumPartitions) + } } /** @@ -3330,7 +3349,7 @@ class Dataset[T] private[sql] ( * @since 3.5.0 */ def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getWithWatermarkBuilder .setInput(plan.getRoot) .setEventTime(eventTime) @@ -3338,14 +3357,28 @@ class Dataset[T] private[sql] ( } } - def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = - sparkSession.newDataset(agnosticEncoder) { + def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = { + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getCollectMetricsBuilder .setInput(plan.getRoot) .setName(name) .addAllMetrics((expr +: exprs).map(_.expr).asJava) } + } + + def observe(observation: Observation, expr: Column, exprs: Column*): Dataset[T] = { + observationsOpt match { + case Some(obs) => + if (obs.contains(observation.name)) { + throw new IllegalArgumentException(s"Observation ${observation.name} already exists.") + } + obs += (observation.name -> observation) + case None => + observationsOpt = Some(mutable.Map(observation.name -> observation)) + } + observe(observation.name, expr, exprs: _*) + } def checkpoint(): Dataset[T] = { throw new UnsupportedOperationException("checkpoint is not implemented.") @@ -3403,10 +3436,24 @@ class Dataset[T] private[sql] ( sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA) } - def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder) + def collectResult(): SparkResult[T] = { + val results = sparkSession.execute(plan, agnosticEncoder) + + results.getObservedMetrics.map { case (name, metric) => + observationsOpt.map { + _.get(name) match { + case Some(observation) => + observation.setMetricsAndNotify(Some(metric)) + case None => + // The observation is not registered using an Observation object. + } + } + } + results + } - def collectObservation(): Seq[ObservedMetrics] = { - sparkSession.execute(plan).flatMap(r => r.getObservedMetricsList().asScala).toSeq + def collectObservations(): Map[String, Map[String, Any]] = { + collectResult().getObservedMetrics } private[sql] def withResult[E](f: SparkResult[T] => E): E = { diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala new file mode 100644 index 000000000000..2ce04bc9e0f6 --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala @@ -0,0 +1,24 @@ +/* + * 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 + +import java.util.UUID + +class Observation(name: String) extends ObservationBase(name) { + def this() = this(UUID.randomUUID().toString) +} diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index e577ef2fc59d..cb2096634942 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -482,17 +482,21 @@ class SparkSession private[sql] ( } } - private[sql] def newDataFrame(f: proto.Relation.Builder => Unit): DataFrame = { - newDataset(UnboundRowEncoder)(f) + private[sql] def newDataFrame( + f: proto.Relation.Builder => Unit, + observations: Option[Map[String, Observation]] = None): DataFrame = { + newDataset(UnboundRowEncoder, observations)(f) } - private[sql] def newDataset[T](encoder: AgnosticEncoder[T])( - f: proto.Relation.Builder => Unit): Dataset[T] = { + private[sql] def newDataset[T]( + encoder: AgnosticEncoder[T], + observations: Option[Map[String, Observation]] = None + )(f: proto.Relation.Builder => Unit): Dataset[T] = { val builder = proto.Relation.newBuilder() f(builder) builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) val plan = proto.Plan.newBuilder().setRoot(builder).build() - new Dataset[T](this, plan, encoder) + new Dataset[T](this, plan, encoder, observations) } @DeveloperApi diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index 62fb316cf142..532b319ae629 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -22,6 +22,8 @@ import java.time.DateTimeException import java.util.Properties import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.DurationInt import scala.jdk.CollectionConverters._ import org.apache.commons.io.FileUtils @@ -41,6 +43,7 @@ import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.test.{IntegrationTestUtils, RemoteSparkSession, SQLHelper} import org.apache.spark.sql.test.SparkConnectServerUtils.port import org.apache.spark.sql.types._ +import org.apache.spark.util.SparkThreadUtils class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateMethodTester { @@ -321,11 +324,6 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM .load() val outputFolderPath = Files.createTempDirectory("output").toAbsolutePath - val dd = - df.observe("observation", min("age"), max("age"), sum("age")) - - val ob = dd.collectObservation() - df.write .format("csv") .mode("overwrite") @@ -1513,6 +1511,40 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM (0 until 5).foreach(i => assert(row.get(i * 2) === row.get(i * 2 + 1))) } } + + test("Observable metrics") { + val df = spark.range(99).withColumn("extra", col("id") - 1) + val ob1 = new Observation("ob1") + val observedDf = df.observe(ob1, min("id"), avg("id"), max("id")) + val observedObservedDf = observedDf.observe("ob2", min("extra"), avg("extra"), max("extra")) + + val ob1Metrics = Map("ob1" -> Map("min(id)" -> 0, "avg(id)" -> 49, "max(id)" -> 98)) + val ob2Metrics = Map("ob2" -> Map("min(extra)" -> -1, "avg(extra)" -> 48, "max(extra)" -> 97)) + + assert(df.collectObservations() === Map.empty) + assert(observedDf.collectObservations() === ob1Metrics) + assert(observedObservedDf.collectObservations() === ob1Metrics ++ ob2Metrics) + } + + test("Observation.get is blocked until the query is finished") { + val df = spark.range(99).withColumn("extra", col("id") - 1) + val observation = new Observation("ob1") + val observedDf = df.observe(observation, min("id"), avg("id"), max("id")) + + // Start a new thread to get the observation + val future = Future { + observation.get + }(ExecutionContext.global) + // make sure the thread in `future` is blocked right now + val e = intercept[java.util.concurrent.TimeoutException] { + SparkThreadUtils.awaitResult(future, 5.seconds) + } + assert(e.getMessage.contains("Future timed out")) + observedDf.collect() + // make sure the thread in `future` is unblocked after the query is finished + val metrics = SparkThreadUtils.awaitResult(future, 5.seconds) + assert(metrics === Map("min(id)" -> 0, "avg(id)" -> 49, "max(id)" -> 98)) + } } private[sql] case class ClassData(a: String, b: Int) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index 7a7c6a2d6c92..ecd610ab9567 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -20,16 +20,18 @@ import java.lang.ref.Cleaner import java.util.Objects import scala.collection.mutable +import scala.jdk.CollectionConverters._ import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch} import org.apache.arrow.vector.types.pojo import org.apache.spark.connect.proto +import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator} -import org.apache.spark.sql.connect.common.DataTypeProtoConverter +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter} import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.ArrowUtils @@ -46,6 +48,7 @@ private[sql] class SparkResult[T]( private[this] var arrowSchema: pojo.Schema = _ private[this] var nextResultIndex: Int = 0 private val resultMap = mutable.Map.empty[Int, (Long, Seq[ArrowMessage])] + private val obMetrics = mutable.ListBuffer.empty[ObservedMetrics] private val cleanable = SparkResult.cleaner.register(this, new SparkResultCloseable(resultMap, responses)) @@ -84,6 +87,9 @@ private[sql] class SparkResult[T]( while (!stop && responses.hasNext) { val response = responses.next() + // Collect metrics for this response + obMetrics ++= response.getObservedMetricsList.asScala + // Save and validate operationId if (opId == null) { opId = response.getOperationId @@ -209,6 +215,24 @@ private[sql] class SparkResult[T]( result } + /** + * Returns all observed metrics of this result. + */ + def getObservedMetrics: Map[String, Map[String, Any]] = { + // We need to process all responses to collect all metrics. + processResponses() + + this.obMetrics.map { metric => + assert(metric.getKeysCount == metric.getValuesCount) + val kv = (0 until metric.getKeysCount).map { i => + val key = metric.getKeys(i) + val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i)) + key -> value + } + metric.getName -> kv.toMap + }.toMap + } + /** * Returns an iterator over the contents of the result. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/ObservationBase.scala b/sql/api/src/main/scala/org/apache/spark/sql/ObservationBase.scala new file mode 100644 index 000000000000..4789ae8975d1 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/ObservationBase.scala @@ -0,0 +1,113 @@ +/* + * 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 + +import scala.jdk.CollectionConverters.MapHasAsJava + +/** + * Helper class to simplify usage of `Dataset.observe(String, Column, Column*)`: + * + * {{{ + * // Observe row count (rows) and highest id (maxid) in the Dataset while writing it + * val observation = Observation("my metrics") + * val observed_ds = ds.observe(observation, count(lit(1)).as("rows"), max($"id").as("maxid")) + * observed_ds.write.parquet("ds.parquet") + * val metrics = observation.get + * }}} + * + * This collects the metrics while the first action is executed on the observed dataset. Subsequent + * actions do not modify the metrics returned by [[get]]. Retrieval of the metric via [[get]] + * blocks until the first action has finished and metrics become available. + * + * This class does not support streaming datasets. + * + * @param name name of the metric + * @since 3.3.0 + */ +abstract class ObservationBase(val name: String) { + + if (name.isEmpty) throw new IllegalArgumentException("Name must not be empty") + + @volatile protected var metrics: Option[Map[String, Any]] = None + + /** + * (Scala-specific) Get the observed metrics. This waits for the observed dataset to finish + * its first action. Only the result of the first action is available. Subsequent actions do not + * modify the result. + * + * @return the observed metrics as a `Map[String, Any]` + * @throws InterruptedException interrupted while waiting + */ + @throws[InterruptedException] + def get: Map[String, _] = { + synchronized { + // we need to loop as wait might return without us calling notify + // https://en.wikipedia.org/w/index.php?title=Spurious_wakeup&oldid=992601610 + while (this.metrics.isEmpty) { + wait() + } + } + + this.metrics.get + } + + /** + * (Java-specific) Get the observed metrics. This waits for the observed dataset to finish + * its first action. Only the result of the first action is available. Subsequent actions do not + * modify the result. + * + * @return the observed metrics as a `java.util.Map[String, Object]` + * @throws InterruptedException interrupted while waiting + */ + @throws[InterruptedException] + def getAsJava: java.util.Map[String, AnyRef] = { + get.map { case (key, value) => (key, value.asInstanceOf[Object]) }.asJava + } + + /** + * Get the observed metrics. This returns the metrics if they are available, otherwise an empty. + * + * @return the observed metrics as a `Map[String, Any]` + */ + @throws[InterruptedException] + private[sql] def getOrEmpty: Map[String, _] = { + synchronized { + if (metrics.isEmpty) { + wait(100) // Wait for 100ms to see if metrics are available + } + metrics.getOrElse(Map.empty) + } + } + + /** + * Set the observed metrics and notify all waiting threads to resume. + * + * @return `true` if all waiting threads were notified, `false` if otherwise. + */ + private[spark] def setMetricsAndNotify(metrics: Option[Map[String, Any]]): Boolean = { + synchronized { + this.metrics = metrics + if(metrics.isDefined) { + notifyAll() + true + } else { + false + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala b/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala index 104e7c101fd1..30d5943c6092 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Observation.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql import java.util.UUID -import scala.jdk.CollectionConverters.MapHasAsJava - import org.apache.spark.sql.catalyst.plans.logical.CollectMetrics import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.util.QueryExecutionListener @@ -47,9 +45,7 @@ import org.apache.spark.util.ArrayImplicits._ * @param name name of the metric * @since 3.3.0 */ -class Observation(val name: String) { - - if (name.isEmpty) throw new IllegalArgumentException("Name must not be empty") +class Observation(name: String) extends ObservationBase(name) { /** * Create an Observation instance without providing a name. This generates a random name. @@ -60,8 +56,6 @@ class Observation(val name: String) { @volatile private var dataframeId: Option[(SparkSession, Long)] = None - @volatile private var metrics: Option[Map[String, Any]] = None - /** * Attach this observation to the given [[Dataset]] to observe aggregation expressions. * @@ -83,55 +77,6 @@ class Observation(val name: String) { ds.observe(name, expr, exprs: _*) } - /** - * (Scala-specific) Get the observed metrics. This waits for the observed dataset to finish - * its first action. Only the result of the first action is available. Subsequent actions do not - * modify the result. - * - * @return the observed metrics as a `Map[String, Any]` - * @throws InterruptedException interrupted while waiting - */ - @throws[InterruptedException] - def get: Map[String, _] = { - synchronized { - // we need to loop as wait might return without us calling notify - // https://en.wikipedia.org/w/index.php?title=Spurious_wakeup&oldid=992601610 - while (this.metrics.isEmpty) { - wait() - } - } - - this.metrics.get - } - - /** - * (Java-specific) Get the observed metrics. This waits for the observed dataset to finish - * its first action. Only the result of the first action is available. Subsequent actions do not - * modify the result. - * - * @return the observed metrics as a `java.util.Map[String, Object]` - * @throws InterruptedException interrupted while waiting - */ - @throws[InterruptedException] - def getAsJava: java.util.Map[String, AnyRef] = { - get.map { case (key, value) => (key, value.asInstanceOf[Object])}.asJava - } - - /** - * Get the observed metrics. This returns the metrics if they are available, otherwise an empty. - * - * @return the observed metrics as a `Map[String, Any]` - */ - @throws[InterruptedException] - private[sql] def getOrEmpty: Map[String, _] = { - synchronized { - if (metrics.isEmpty) { - wait(100) // Wait for 100ms to see if metrics are available - } - metrics.getOrElse(Map.empty) - } - } - private[sql] def register(sparkSession: SparkSession, dataframeId: Long): Unit = { // makes this class thread-safe: // only the first thread entering this block can set sparkSession @@ -158,9 +103,8 @@ class Observation(val name: String) { case _ => false }) { val row = qe.observedMetrics.get(name) - this.metrics = row.map(r => r.getValuesMap[Any](r.schema.fieldNames.toImmutableArraySeq)) - if (metrics.isDefined) { - notifyAll() + val metrics = row.map(r => r.getValuesMap[Any](r.schema.fieldNames.toImmutableArraySeq)) + if (setMetricsAndNotify(metrics)) { unregister() } } From 3441c8ae83cfca2cddd1b672bdc7a4c24a48dc25 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Mon, 25 Mar 2024 15:29:16 +0100 Subject: [PATCH 03/20] . --- .../org/apache/spark/sql/ClientE2ETestSuite.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index 532b319ae629..7efd0e2fac9c 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -1532,17 +1532,15 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM val observedDf = df.observe(observation, min("id"), avg("id"), max("id")) // Start a new thread to get the observation - val future = Future { - observation.get - }(ExecutionContext.global) - // make sure the thread in `future` is blocked right now + val future = Future(observation.get)(ExecutionContext.global) + // make sure the thread is blocked right now val e = intercept[java.util.concurrent.TimeoutException] { - SparkThreadUtils.awaitResult(future, 5.seconds) + SparkThreadUtils.awaitResult(future, 2.seconds) } assert(e.getMessage.contains("Future timed out")) observedDf.collect() - // make sure the thread in `future` is unblocked after the query is finished - val metrics = SparkThreadUtils.awaitResult(future, 5.seconds) + // make sure the thread is unblocked after the query is finished + val metrics = SparkThreadUtils.awaitResult(future, 2.seconds) assert(metrics === Map("min(id)" -> 0, "avg(id)" -> 49, "max(id)" -> 98)) } } From baa5ca3cf614c95d656e679c0cad9543761c160a Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Mon, 25 Mar 2024 17:07:27 +0100 Subject: [PATCH 04/20] try fix tests --- .../scala/org/apache/spark/sql/Dataset.scala | 21 +++++----- .../org/apache/spark/sql/SparkSession.scala | 4 +- .../sql/connect/client/SparkResult.scala | 38 +++++++++---------- 3 files changed, 29 insertions(+), 34 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index bd64b4e4a325..c620c08801db 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -147,7 +147,8 @@ class Dataset[T] private[sql] ( } } - private def getObservationsMapOpt: Option[Map[String, Observation]] = observationsOpt.map(_.toMap) + private def getObservationsMapOpt: Option[Map[String, Observation]] = + observationsOpt.map(_.toMap) override def toString: String = { try { @@ -1285,9 +1286,8 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def filter(condition: Column): Dataset[T] = - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { - builder => - builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) } /** @@ -3358,12 +3358,11 @@ class Dataset[T] private[sql] ( } def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { - builder => - builder.getCollectMetricsBuilder - .setInput(plan.getRoot) - .setName(name) - .addAllMetrics((expr +: exprs).map(_.expr).asJava) + sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + builder.getCollectMetricsBuilder + .setInput(plan.getRoot) + .setName(name) + .addAllMetrics((expr +: exprs).map(_.expr).asJava) } } @@ -3439,7 +3438,7 @@ class Dataset[T] private[sql] ( def collectResult(): SparkResult[T] = { val results = sparkSession.execute(plan, agnosticEncoder) - results.getObservedMetrics.map { case (name, metric) => + results.observedMetrics.map { case (name, metric) => observationsOpt.map { _.get(name) match { case Some(observation) => diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index cb2096634942..8a55fa935ddf 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -490,8 +490,8 @@ class SparkSession private[sql] ( private[sql] def newDataset[T]( encoder: AgnosticEncoder[T], - observations: Option[Map[String, Observation]] = None - )(f: proto.Relation.Builder => Unit): Dataset[T] = { + observations: Option[Map[String, Observation]] = None)( + f: proto.Relation.Builder => Unit): Dataset[T] = { val builder = proto.Relation.newBuilder() f(builder) builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index ecd610ab9567..a9f22a39b735 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -20,7 +20,6 @@ import java.lang.ref.Cleaner import java.util.Objects import scala.collection.mutable -import scala.jdk.CollectionConverters._ import org.apache.arrow.memory.BufferAllocator import org.apache.arrow.vector.ipc.message.{ArrowMessage, ArrowRecordBatch} @@ -48,7 +47,7 @@ private[sql] class SparkResult[T]( private[this] var arrowSchema: pojo.Schema = _ private[this] var nextResultIndex: Int = 0 private val resultMap = mutable.Map.empty[Int, (Long, Seq[ArrowMessage])] - private val obMetrics = mutable.ListBuffer.empty[ObservedMetrics] + private[spark] val observedMetrics = mutable.Map.empty[String, Map[String, Any]] private val cleanable = SparkResult.cleaner.register(this, new SparkResultCloseable(resultMap, responses)) @@ -88,7 +87,7 @@ private[sql] class SparkResult[T]( val response = responses.next() // Collect metrics for this response - obMetrics ++= response.getObservedMetricsList.asScala + observedMetrics ++= processObservedMetrics(response.getObservedMetricsList) // Save and validate operationId if (opId == null) { @@ -165,6 +164,21 @@ private[sql] class SparkResult[T]( nonEmpty } + private def processObservedMetrics( + metrics: java.util.List[ObservedMetrics]): Iterable[(String, Map[String, Any])] = { + val processed = mutable.ListBuffer.empty[(String, Map[String, Any])] + metrics.forEach { metric => + assert(metric.getKeysCount == metric.getValuesCount) + val kv = (0 until metric.getKeysCount).map { i => + val key = metric.getKeys(i) + val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i)) + key -> value + } + processed += metric.getName -> kv.toMap + } + processed + } + /** * Returns the number of elements in the result. */ @@ -215,24 +229,6 @@ private[sql] class SparkResult[T]( result } - /** - * Returns all observed metrics of this result. - */ - def getObservedMetrics: Map[String, Map[String, Any]] = { - // We need to process all responses to collect all metrics. - processResponses() - - this.obMetrics.map { metric => - assert(metric.getKeysCount == metric.getValuesCount) - val kv = (0 until metric.getKeysCount).map { i => - val key = metric.getKeys(i) - val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i)) - key -> value - } - metric.getName -> kv.toMap - }.toMap - } - /** * Returns an iterator over the contents of the result. */ From 8efa252b0ead9386106b4daf5a77b2946f971529 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Mon, 25 Mar 2024 17:25:25 +0100 Subject: [PATCH 05/20] oops --- .../jvm/src/main/scala/org/apache/spark/sql/Dataset.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index c620c08801db..4a48fde4738e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -3452,7 +3452,7 @@ class Dataset[T] private[sql] ( } def collectObservations(): Map[String, Map[String, Any]] = { - collectResult().getObservedMetrics + collectResult().observedMetrics.toMap } private[sql] def withResult[E](f: SparkResult[T] => E): E = { From 13eadb35b60da239fa4df37e8e7da116ab96103c Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Tue, 26 Mar 2024 14:30:25 +0100 Subject: [PATCH 06/20] avoid collecting all results --- .../scala/org/apache/spark/sql/Dataset.scala | 22 ++++--------------- .../org/apache/spark/sql/SparkSession.scala | 7 ++++-- .../sql/connect/client/SparkResult.scala | 22 ++++++++++++++++--- 3 files changed, 28 insertions(+), 23 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 4a48fde4738e..86107148cf5d 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -3435,25 +3435,11 @@ class Dataset[T] private[sql] ( sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA) } - def collectResult(): SparkResult[T] = { - val results = sparkSession.execute(plan, agnosticEncoder) - - results.observedMetrics.map { case (name, metric) => - observationsOpt.map { - _.get(name) match { - case Some(observation) => - observation.setMetricsAndNotify(Some(metric)) - case None => - // The observation is not registered using an Observation object. - } - } - } - results - } + def collectResult(): SparkResult[T] = + sparkSession.execute(plan, agnosticEncoder, getObservationsMapOpt) - def collectObservations(): Map[String, Map[String, Any]] = { - collectResult().observedMetrics.toMap - } + def collectObservations(): Map[String, Map[String, Any]] = + collectResult().getObservedMetrics private[sql] def withResult[E](f: SparkResult[T] => E): E = { val result = collectResult() diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 8a55fa935ddf..ef3167510743 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -542,9 +542,12 @@ class SparkSession private[sql] ( private[sql] def timeZoneId: String = conf.get(SqlApiConf.SESSION_LOCAL_TIMEZONE_KEY) - private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { + private[sql] def execute[T]( + plan: proto.Plan, + encoder: AgnosticEncoder[T], + observationsOpt: Option[Map[String, Observation]] = None): SparkResult[T] = { val value = client.execute(plan) - val result = new SparkResult(value, allocator, encoder, timeZoneId) + val result = new SparkResult(value, allocator, encoder, timeZoneId, observationsOpt) result } diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index a9f22a39b735..d113e16c2fb6 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -27,6 +27,7 @@ import org.apache.arrow.vector.types.pojo import org.apache.spark.connect.proto import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics +import org.apache.spark.sql.ObservationBase import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator} @@ -38,7 +39,8 @@ private[sql] class SparkResult[T]( responses: CloseableIterator[proto.ExecutePlanResponse], allocator: BufferAllocator, encoder: AgnosticEncoder[T], - timeZoneId: String) + timeZoneId: String, + observationsOpt: Option[Map[String, ObservationBase]] = None) extends AutoCloseable { self => private[this] var opId: String = _ @@ -47,7 +49,7 @@ private[sql] class SparkResult[T]( private[this] var arrowSchema: pojo.Schema = _ private[this] var nextResultIndex: Int = 0 private val resultMap = mutable.Map.empty[Int, (Long, Seq[ArrowMessage])] - private[spark] val observedMetrics = mutable.Map.empty[String, Map[String, Any]] + private val observedMetrics = mutable.Map.empty[String, Map[String, Any]] private val cleanable = SparkResult.cleaner.register(this, new SparkResultCloseable(resultMap, responses)) @@ -173,8 +175,13 @@ private[sql] class SparkResult[T]( val key = metric.getKeys(i) val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i)) key -> value + }.toMap + processed += metric.getName -> kv + // If the metrics is registered by an Observation object, attach them and unblock any + // blocked thread. + observationsOpt.map { observations => + observations.get(metric.getName).map(_.setMetricsAndNotify(Some(kv))) } - processed += metric.getName -> kv.toMap } processed } @@ -229,6 +236,15 @@ private[sql] class SparkResult[T]( result } + /** + * Returns all observed metrics in the result. + */ + def getObservedMetrics: Map[String, Map[String, Any]] = { + // We need to process all responses to get all metrics. + processResponses() + observedMetrics.toMap + } + /** * Returns an iterator over the contents of the result. */ From 2d0af648b072bd86ec8f4e85a3303665237fb680 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Tue, 26 Mar 2024 16:30:54 +0100 Subject: [PATCH 07/20] mima --- .../scala/org/apache/spark/sql/Dataset.scala | 4 ++-- .../org/apache/spark/sql/Observation.scala | 22 +++++++++++++++++++ .../org/apache/spark/sql/SparkSession.scala | 8 +++---- 3 files changed, 28 insertions(+), 6 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 86107148cf5d..e09e2f92e90c 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -132,7 +132,7 @@ class Dataset[T] private[sql] ( val sparkSession: SparkSession, @DeveloperApi val plan: proto.Plan, val encoder: Encoder[T], - @DeveloperApi carryOverObservations: Option[Map[String, Observation]] = None) + @DeveloperApi carryOverObservationsOpt: Option[Map[String, Observation]] = None) extends Serializable { // Make sure we don't forget to set plan id. assert(plan.getRoot.getCommon.hasPlanId) @@ -140,7 +140,7 @@ class Dataset[T] private[sql] ( private[sql] val agnosticEncoder: AgnosticEncoder[T] = encoderFor(encoder) private var observationsOpt: Option[mutable.Map[String, Observation]] = { - carryOverObservations match { + carryOverObservationsOpt match { case Some(observations) => Some(mutable.Map.newBuilder[String, Observation].addAll(observations).result()) case None => None diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala index 2ce04bc9e0f6..366144dd94cb 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala @@ -20,5 +20,27 @@ package org.apache.spark.sql import java.util.UUID class Observation(name: String) extends ObservationBase(name) { + + /** + * Create an Observation instance without providing a name. This generates a random name. + */ def this() = this(UUID.randomUUID().toString) } + +/** + * (Scala-specific) Create instances of Observation via Scala `apply`. + * @since 3.3.0 + */ +object Observation { + + /** + * Observation constructor for creating an anonymous observation. + */ + def apply(): Observation = new Observation() + + /** + * Observation constructor for creating a named observation. + */ + def apply(name: String): Observation = new Observation(name) + +} diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index ef3167510743..fc7489ee70c7 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -484,19 +484,19 @@ class SparkSession private[sql] ( private[sql] def newDataFrame( f: proto.Relation.Builder => Unit, - observations: Option[Map[String, Observation]] = None): DataFrame = { - newDataset(UnboundRowEncoder, observations)(f) + observationsOpt: Option[Map[String, Observation]] = None): DataFrame = { + newDataset(UnboundRowEncoder, observationsOpt)(f) } private[sql] def newDataset[T]( encoder: AgnosticEncoder[T], - observations: Option[Map[String, Observation]] = None)( + observationsOpt: Option[Map[String, Observation]] = None)( f: proto.Relation.Builder => Unit): Dataset[T] = { val builder = proto.Relation.newBuilder() f(builder) builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) val plan = proto.Plan.newBuilder().setRoot(builder).build() - new Dataset[T](this, plan, encoder, observations) + new Dataset[T](this, plan, encoder, observationsOpt) } @DeveloperApi From 103e57ae3f357b2ad7042d49480bd0611bcbde3d Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Wed, 27 Mar 2024 10:12:14 +0100 Subject: [PATCH 08/20] mima again --- .../sql/connect/client/CheckConnectJvmClientCompatibility.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index bd5ff6af7464..e870c1e73ce3 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -361,6 +361,8 @@ object CheckConnectJvmClientCompatibility { ), // developer API ProblemFilters.exclude[DirectMissingMethodProblem]( "org.apache.spark.sql.Dataset.collectResult"), + ProblemFilters.exclude[DirectMissingMethodProblem]( + "org.apache.spark.sql.Dataset.collectObservations"), // RuntimeConfig ProblemFilters.exclude[MissingTypesProblem]( From 7ce7a57872fd4ca7af528a09cc3ba2fd59ed5b38 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Tue, 2 Apr 2024 10:57:57 +0200 Subject: [PATCH 09/20] docstring --- .../scala/org/apache/spark/sql/Dataset.scala | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index e09e2f92e90c..703854b47bcc 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -3357,6 +3357,36 @@ class Dataset[T] private[sql] ( } } + /** + * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset + * that returns the same result as the input, with the following guarantees: + *
    + *
  • It will compute the defined aggregates (metrics) on all the data that is flowing through + * the Dataset at that point.
  • + *
  • It will report the value of the defined aggregate columns as soon as we reach a + * completion point. A completion point is currently defined as the end of a query.
  • + *
+ * Please note that continuous execution is currently not supported. + * + * The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or + * more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that + * contain references to the input Dataset's columns must always be wrapped in an aggregate + * function. + * + * A user can retrieve the metrics by calling + * `org.apache.spark.sql.Dataset.collectObservations()`. + * + * {{{ + * // Observe row count (rows) and highest id (maxid) in the Dataset while writing it + * val observed_ds = ds.observe("my_metrics", count(lit(1)).as("rows"), max($"id").as("maxid")) + * observed_ds.write.parquet("ds.parquet") + * val metrics = observed_ds.collectObservations() + * }}} + * + * @group typedrel + * @since 4.0.0 + */ + @scala.annotation.varargs def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = { sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => builder.getCollectMetricsBuilder @@ -3366,6 +3396,29 @@ class Dataset[T] private[sql] ( } } + /** + * Observe (named) metrics through an `org.apache.spark.sql.Observation` instance. + * This is equivalent to calling `observe(String, Column, Column*)` but does not require to + * collect all results before returning the metrics - the metrics are filled during iterating + * the results, as soon as they are available. + * This method does not support streaming datasets. + * + * A user can retrieve the metrics by accessing `org.apache.spark.sql.Observation.get`. + * + * {{{ + * // Observe row count (rows) and highest id (maxid) in the Dataset while writing it + * val observation = Observation("my_metrics") + * val observed_ds = ds.observe(observation, count(lit(1)).as("rows"), max($"id").as("maxid")) + * observed_ds.write.parquet("ds.parquet") + * val metrics = observation.get + * }}} + * + * @throws IllegalArgumentException If this is a streaming Dataset (this.isStreaming == true) + * + * @group typedrel + * @since 4.0.0 + */ + @scala.annotation.varargs def observe(observation: Observation, expr: Column, exprs: Column*): Dataset[T] = { observationsOpt match { case Some(obs) => From cf3437beb0211d0240f8010dce6ec9a2c2c1d68a Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Thu, 11 Apr 2024 12:42:25 +0200 Subject: [PATCH 10/20] move to plan_id --- .../scala/org/apache/spark/sql/Dataset.scala | 73 +++++++------------ .../org/apache/spark/sql/Observation.scala | 22 ++++++ .../org/apache/spark/sql/SparkSession.scala | 16 ++-- .../CheckConnectJvmClientCompatibility.scala | 3 - .../main/protobuf/spark/connect/base.proto | 1 + .../sql/connect/client/SparkResult.scala | 4 +- .../execution/ExecuteThreadRunner.scala | 1 + .../execution/SparkConnectPlanExecution.scala | 9 ++- .../execution/ReattachableExecuteSuite.scala | 11 +++ 9 files changed, 80 insertions(+), 60 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 703854b47bcc..278ebdac105e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -131,25 +131,13 @@ import org.apache.spark.util.SparkClassUtils class Dataset[T] private[sql] ( val sparkSession: SparkSession, @DeveloperApi val plan: proto.Plan, - val encoder: Encoder[T], - @DeveloperApi carryOverObservationsOpt: Option[Map[String, Observation]] = None) + val encoder: Encoder[T]) extends Serializable { // Make sure we don't forget to set plan id. assert(plan.getRoot.getCommon.hasPlanId) private[sql] val agnosticEncoder: AgnosticEncoder[T] = encoderFor(encoder) - private var observationsOpt: Option[mutable.Map[String, Observation]] = { - carryOverObservationsOpt match { - case Some(observations) => - Some(mutable.Map.newBuilder[String, Observation].addAll(observations).result()) - case None => None - } - } - - private def getObservationsMapOpt: Option[Map[String, Observation]] = - observationsOpt.map(_.toMap) - override def toString: String = { try { val builder = new mutable.StringBuilder @@ -548,7 +536,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def show(numRows: Int, truncate: Int, vertical: Boolean): Unit = { - val df = sparkSession.newDataset(StringEncoder, getObservationsMapOpt) { builder => + val df = sparkSession.newDataset(StringEncoder) { builder => builder.getShowStringBuilder .setInput(plan.getRoot) .setNumRows(numRows) @@ -856,7 +844,7 @@ class Dataset[T] private[sql] ( } private def buildSort(global: Boolean, sortExprs: Seq[Column]): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getSortBuilder .setInput(plan.getRoot) .setIsGlobal(global) @@ -910,7 +898,7 @@ class Dataset[T] private[sql] ( EncoderField(s"_2", other.agnosticEncoder, rightNullable, Metadata.empty)), None) - sparkSession.newDataset(tupleEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(tupleEncoder) { builder => val joinBuilder = builder.getJoinBuilder joinBuilder .setLeft(plan.getRoot) @@ -1040,7 +1028,7 @@ class Dataset[T] private[sql] ( */ @scala.annotation.varargs def hint(name: String, parameters: Any*): Dataset[T] = - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getHintBuilder .setInput(plan.getRoot) .setName(name) @@ -1102,7 +1090,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def as(alias: String): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getSubqueryAliasBuilder .setInput(plan.getRoot) .setAlias(alias) @@ -1197,7 +1185,7 @@ class Dataset[T] private[sql] ( } else { c1.expr } - sparkSession.newDataset(encoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(encoder) { builder => builder.getProjectBuilder .setInput(plan.getRoot) .addExpressions(expr) @@ -1219,7 +1207,7 @@ class Dataset[T] private[sql] ( * methods and typed select methods is the encoder used to build the return dataset. */ private def selectUntyped(encoder: AgnosticEncoder[_], cols: Seq[Column]): Dataset[_] = { - sparkSession.newDataset(encoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(encoder) { builder => builder.getProjectBuilder .setInput(plan.getRoot) .addAllExpressions(cols.map(_.expr).asJava) @@ -1286,7 +1274,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def filter(condition: Column): Dataset[T] = - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) } @@ -1408,7 +1396,7 @@ class Dataset[T] private[sql] ( val reduceExpr = Column.fn("reduce", udf.apply(col("*"), col("*"))).expr val result = sparkSession - .newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + .newDataset(agnosticEncoder) { builder => builder.getAggregateBuilder .setInput(plan.getRoot) .addAggregateExpressions(reduceExpr) @@ -1802,7 +1790,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def limit(n: Int): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getLimitBuilder .setInput(plan.getRoot) .setLimit(n) @@ -1816,7 +1804,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def offset(n: Int): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getOffsetBuilder .setInput(plan.getRoot) .setOffset(n) @@ -1826,7 +1814,7 @@ class Dataset[T] private[sql] ( private def buildSetOp(right: Dataset[T], setOpType: proto.SetOperation.SetOpType)( f: proto.SetOperation.Builder => Unit): Dataset[T] = { checkSameSparkSession(right) - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => f( builder.getSetOpBuilder .setSetOpType(setOpType) @@ -2099,7 +2087,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getSampleBuilder .setInput(plan.getRoot) .setWithReplacement(withReplacement) @@ -2167,7 +2155,7 @@ class Dataset[T] private[sql] ( normalizedCumWeights .sliding(2) .map { case Array(low, high) => - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getSampleBuilder .setInput(sortedInput) .setWithReplacement(false) @@ -2495,7 +2483,7 @@ class Dataset[T] private[sql] ( private def buildDropDuplicates( columns: Option[Seq[String]], withinWaterMark: Boolean): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => val dropBuilder = builder.getDeduplicateBuilder .setInput(plan.getRoot) .setWithinWatermark(withinWaterMark) @@ -2728,7 +2716,7 @@ class Dataset[T] private[sql] ( function = func, inputEncoders = agnosticEncoder :: Nil, outputEncoder = PrimitiveBooleanEncoder) - sparkSession.newDataset[T](agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset[T](agnosticEncoder) { builder => builder.getFilterBuilder .setInput(plan.getRoot) .setCondition(udf.apply(col("*")).expr) @@ -2781,7 +2769,7 @@ class Dataset[T] private[sql] ( function = func, inputEncoders = agnosticEncoder :: Nil, outputEncoder = outputEncoder) - sparkSession.newDataset(outputEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(outputEncoder) { builder => builder.getMapPartitionsBuilder .setInput(plan.getRoot) .setFunc(udf.apply(col("*")).expr.getCommonInlineUserDefinedFunction) @@ -2949,7 +2937,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def tail(n: Int): Array[T] = { - val lastN = sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + val lastN = sparkSession.newDataset(agnosticEncoder) { builder => builder.getTailBuilder .setInput(plan.getRoot) .setLimit(n) @@ -3020,7 +3008,7 @@ class Dataset[T] private[sql] ( } private def buildRepartition(numPartitions: Int, shuffle: Boolean): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getRepartitionBuilder .setInput(plan.getRoot) .setNumPartitions(numPartitions) @@ -3031,7 +3019,7 @@ class Dataset[T] private[sql] ( private def buildRepartitionByExpression( numPartitions: Option[Int], partitionExprs: Seq[Column]): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => val repartitionBuilder = builder.getRepartitionByExpressionBuilder .setInput(plan.getRoot) .addAllPartitionExprs(partitionExprs.map(_.expr).asJava) @@ -3349,7 +3337,7 @@ class Dataset[T] private[sql] ( * @since 3.5.0 */ def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getWithWatermarkBuilder .setInput(plan.getRoot) .setEventTime(eventTime) @@ -3388,7 +3376,7 @@ class Dataset[T] private[sql] ( */ @scala.annotation.varargs def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder, getObservationsMapOpt) { builder => + sparkSession.newDataset(agnosticEncoder) { builder => builder.getCollectMetricsBuilder .setInput(plan.getRoot) .setName(name) @@ -3420,16 +3408,9 @@ class Dataset[T] private[sql] ( */ @scala.annotation.varargs def observe(observation: Observation, expr: Column, exprs: Column*): Dataset[T] = { - observationsOpt match { - case Some(obs) => - if (obs.contains(observation.name)) { - throw new IllegalArgumentException(s"Observation ${observation.name} already exists.") - } - obs += (observation.name -> observation) - case None => - observationsOpt = Some(mutable.Map(observation.name -> observation)) - } - observe(observation.name, expr, exprs: _*) + val df = observe(observation.name, expr, exprs: _*) + observation.register(sparkSession, df.getPlanId.get) + df } def checkpoint(): Dataset[T] = { @@ -3489,7 +3470,7 @@ class Dataset[T] private[sql] ( } def collectResult(): SparkResult[T] = - sparkSession.execute(plan, agnosticEncoder, getObservationsMapOpt) + sparkSession.execute(plan, agnosticEncoder, Some(sparkSession.observationRegistry.toMap)) def collectObservations(): Map[String, Map[String, Any]] = collectResult().getObservedMetrics diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala index 366144dd94cb..1cf8d5b772cb 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala @@ -25,6 +25,28 @@ class Observation(name: String) extends ObservationBase(name) { * Create an Observation instance without providing a name. This generates a random name. */ def this() = this(UUID.randomUUID().toString) + + @volatile private var planId: Option[(SparkSession, Long)] = None + + private[sql] def register(sparkSession: SparkSession, planId: Long): Unit = { + // makes this class thread-safe: + // only the first thread entering this block can set sparkSession + // all other threads will see the exception, as it is only allowed to do this once + synchronized { + if (this.planId.isDefined) { + throw new IllegalArgumentException("An Observation can be used with a Dataset only once") + } + this.planId = Some((sparkSession, planId)) + } + + sparkSession.observationRegistry.put(planId, this) + } + + private def unregister(): Unit = { + this.planId.map { case (sparkSession, planId) => + sparkSession.observationRegistry.remove(planId) + } + } } /** diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index fc7489ee70c7..afe199d21ffc 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -21,6 +21,7 @@ import java.net.URI import java.util.concurrent.TimeUnit._ import java.util.concurrent.atomic.{AtomicLong, AtomicReference} +import scala.collection.mutable import scala.jdk.CollectionConverters._ import scala.reflect.runtime.universe.TypeTag @@ -80,6 +81,8 @@ class SparkSession private[sql] ( client.analyze(proto.AnalyzePlanRequest.AnalyzeCase.SPARK_VERSION).getSparkVersion.getVersion } + private[sql] val observationRegistry = mutable.Map.empty[Long, Observation] + /** * Runtime configuration interface for Spark. * @@ -482,21 +485,18 @@ class SparkSession private[sql] ( } } - private[sql] def newDataFrame( - f: proto.Relation.Builder => Unit, - observationsOpt: Option[Map[String, Observation]] = None): DataFrame = { - newDataset(UnboundRowEncoder, observationsOpt)(f) + private[sql] def newDataFrame(f: proto.Relation.Builder => Unit): DataFrame = { + newDataset(UnboundRowEncoder)(f) } private[sql] def newDataset[T]( - encoder: AgnosticEncoder[T], - observationsOpt: Option[Map[String, Observation]] = None)( + encoder: AgnosticEncoder[T])( f: proto.Relation.Builder => Unit): Dataset[T] = { val builder = proto.Relation.newBuilder() f(builder) builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) val plan = proto.Plan.newBuilder().setRoot(builder).build() - new Dataset[T](this, plan, encoder, observationsOpt) + new Dataset[T](this, plan, encoder) } @DeveloperApi @@ -545,7 +545,7 @@ class SparkSession private[sql] ( private[sql] def execute[T]( plan: proto.Plan, encoder: AgnosticEncoder[T], - observationsOpt: Option[Map[String, Observation]] = None): SparkResult[T] = { + observationsOpt: Option[Map[Long, Observation]] = None): SparkResult[T] = { val value = client.execute(plan) val result = new SparkResult(value, allocator, encoder, timeZoneId, observationsOpt) result diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index e870c1e73ce3..903ddb2687e7 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -194,9 +194,6 @@ object CheckConnectJvmClientCompatibility { ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.COL_POS_KEY"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.DATASET_ID_KEY"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.curId"), - ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.observe"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Observation"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Observation$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.ObservationListener$"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.Dataset.queryExecution"), 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 bcc7edc55550..5e376ee295da 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -428,6 +428,7 @@ message ExecutePlanResponse { string name = 1; repeated Expression.Literal values = 2; repeated string keys = 3; + int64 plan_id = 4; } message ResultComplete { diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index d113e16c2fb6..559943fbe049 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -40,7 +40,7 @@ private[sql] class SparkResult[T]( allocator: BufferAllocator, encoder: AgnosticEncoder[T], timeZoneId: String, - observationsOpt: Option[Map[String, ObservationBase]] = None) + observationsOpt: Option[Map[Long, ObservationBase]] = None) extends AutoCloseable { self => private[this] var opId: String = _ @@ -180,7 +180,7 @@ private[sql] class SparkResult[T]( // If the metrics is registered by an Observation object, attach them and unblock any // blocked thread. observationsOpt.map { observations => - observations.get(metric.getName).map(_.setMetricsAndNotify(Some(kv))) + observations.get(metric.getPlanId).map(_.setMetricsAndNotify(Some(kv))) } } processed diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala index 41146e4ef688..c9e28b9f2311 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala @@ -213,6 +213,7 @@ private[connect] class ExecuteThreadRunner(executeHolder: ExecuteHolder) extends .createObservedMetricsResponse( executeHolder.sessionHolder.sessionId, executeHolder.sessionHolder.serverSessionId, + executeHolder.request.getPlan.getRoot.getCommon.getPlanId, observedMetrics ++ accumulatedInPython)) } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala index 23390bf7aba8..34a99cfb6ebd 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala @@ -253,8 +253,13 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) name -> values } if (observedMetrics.nonEmpty) { + val planId = executeHolder.request.getPlan.getRoot.getCommon.getPlanId Some(SparkConnectPlanExecution - .createObservedMetricsResponse(sessionId, sessionHolder.serverSessionId, observedMetrics)) + .createObservedMetricsResponse( + sessionId, + sessionHolder.serverSessionId, + planId, + observedMetrics)) } else None } } @@ -263,11 +268,13 @@ object SparkConnectPlanExecution { def createObservedMetricsResponse( sessionId: String, serverSessionId: String, + planId: Long, metrics: Map[String, Seq[(Option[String], Any)]]): ExecutePlanResponse = { val observedMetrics = metrics.map { case (name, values) => val metrics = ExecutePlanResponse.ObservedMetrics .newBuilder() .setName(name) + .setPlanId(planId) values.foreach { case (key, value) => metrics.addValues(toLiteralProto(value)) key.foreach(metrics.addKeys) diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala index 25e6cc48a199..067857ccd059 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala @@ -399,6 +399,17 @@ class ReattachableExecuteSuite extends SparkConnectServerTest { } } + test("observations") { + withClient { client => + val p = buildPlan("select * from range(1999)") + val d = p.getRoot.getCommon.getPlanId + val o = proto.Relation.newBuilder() + o.getCommonBuilder.setPlanId(19L) + o.getCollectMetricsBuilder.setName("my_observation").setInput(p.getRoot) + client.execute(buildPlan("select * from range(1)")) + } + } + test("SPARK-46660: reattach updates aliveness of session holder") { withRawBlockingStub { stub => val operationId = UUID.randomUUID().toString From f9cc2a5fb75f362213120d12343437373045bf19 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Thu, 11 Apr 2024 14:02:15 +0200 Subject: [PATCH 11/20] style --- .../scala/org/apache/spark/sql/Dataset.scala | 46 ++++++++----------- .../org/apache/spark/sql/SparkSession.scala | 5 +- .../execution/ReattachableExecuteSuite.scala | 11 ----- 3 files changed, 21 insertions(+), 41 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 278ebdac105e..1861caa6846d 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1089,12 +1089,10 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def as(alias: String): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => - builder.getSubqueryAliasBuilder - .setInput(plan.getRoot) - .setAlias(alias) - } + def as(alias: String): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => + builder.getSubqueryAliasBuilder + .setInput(plan.getRoot) + .setAlias(alias) } /** @@ -1273,10 +1271,10 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def filter(condition: Column): Dataset[T] = - sparkSession.newDataset(agnosticEncoder) { builder => + def filter(condition: Column): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { + builder => builder.getFilterBuilder.setInput(plan.getRoot).setCondition(condition.expr) - } + } /** * Filters rows using the given SQL expression. @@ -1789,12 +1787,10 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def limit(n: Int): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => - builder.getLimitBuilder - .setInput(plan.getRoot) - .setLimit(n) - } + def limit(n: Int): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => + builder.getLimitBuilder + .setInput(plan.getRoot) + .setLimit(n) } /** @@ -1803,12 +1799,10 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def offset(n: Int): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => - builder.getOffsetBuilder - .setInput(plan.getRoot) - .setOffset(n) - } + def offset(n: Int): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => + builder.getOffsetBuilder + .setInput(plan.getRoot) + .setOffset(n) } private def buildSetOp(right: Dataset[T], setOpType: proto.SetOperation.SetOpType)( @@ -2482,8 +2476,8 @@ class Dataset[T] private[sql] ( private def buildDropDuplicates( columns: Option[Seq[String]], - withinWaterMark: Boolean): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => + withinWaterMark: Boolean): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { + builder => val dropBuilder = builder.getDeduplicateBuilder .setInput(plan.getRoot) .setWithinWatermark(withinWaterMark) @@ -2492,7 +2486,6 @@ class Dataset[T] private[sql] ( } else { dropBuilder.setAllColumnsAsKeys(true) } - } } /** @@ -3018,13 +3011,12 @@ class Dataset[T] private[sql] ( private def buildRepartitionByExpression( numPartitions: Option[Int], - partitionExprs: Seq[Column]): Dataset[T] = { - sparkSession.newDataset(agnosticEncoder) { builder => + partitionExprs: Seq[Column]): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { + builder => val repartitionBuilder = builder.getRepartitionByExpressionBuilder .setInput(plan.getRoot) .addAllPartitionExprs(partitionExprs.map(_.expr).asJava) numPartitions.foreach(repartitionBuilder.setNumPartitions) - } } /** diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 860a6e9d23b5..16fc0a5a8e53 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -489,8 +489,7 @@ class SparkSession private[sql] ( newDataset(UnboundRowEncoder)(f) } - private[sql] def newDataset[T]( - encoder: AgnosticEncoder[T])( + private[sql] def newDataset[T](encoder: AgnosticEncoder[T])( f: proto.Relation.Builder => Unit): Dataset[T] = { val builder = proto.Relation.newBuilder() f(builder) @@ -569,7 +568,7 @@ class SparkSession private[sql] ( builder.getCommonBuilder.setPlanId(planIdGenerator.getAndIncrement()) val plan = proto.Plan.newBuilder().setRoot(builder).build() // .foreach forces that the iterator is consumed and closed - execute(plan).foreach(_ => ()) + client.execute(plan).foreach(_ => ()) } private[sql] def execute(command: proto.Command): Seq[ExecutePlanResponse] = { diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala index 067857ccd059..25e6cc48a199 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala @@ -399,17 +399,6 @@ class ReattachableExecuteSuite extends SparkConnectServerTest { } } - test("observations") { - withClient { client => - val p = buildPlan("select * from range(1999)") - val d = p.getRoot.getCommon.getPlanId - val o = proto.Relation.newBuilder() - o.getCommonBuilder.setPlanId(19L) - o.getCollectMetricsBuilder.setName("my_observation").setInput(p.getRoot) - client.execute(buildPlan("select * from range(1)")) - } - } - test("SPARK-46660: reattach updates aliveness of session holder") { withRawBlockingStub { stub => val operationId = UUID.randomUUID().toString From 0437c45b7469458c09d75ba93128197bccb60528 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Thu, 11 Apr 2024 14:14:33 +0200 Subject: [PATCH 12/20] . --- .../jvm/src/main/scala/org/apache/spark/sql/Dataset.scala | 3 +-- .../src/main/scala/org/apache/spark/sql/Observation.scala | 5 +++++ .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 6 ++---- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 1861caa6846d..4eda2699f051 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -3461,8 +3461,7 @@ class Dataset[T] private[sql] ( sparkSession.analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA) } - def collectResult(): SparkResult[T] = - sparkSession.execute(plan, agnosticEncoder, Some(sparkSession.observationRegistry.toMap)) + def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder) def collectObservations(): Map[String, Map[String, Any]] = collectResult().getObservedMetrics diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala index 1cf8d5b772cb..df6e4af06b15 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala @@ -47,6 +47,11 @@ class Observation(name: String) extends ObservationBase(name) { sparkSession.observationRegistry.remove(planId) } } + + override private[spark] def setMetricsAndNotify(metrics: Option[Map[String, Any]]): Boolean = { + this.unregister() + super.setMetricsAndNotify(metrics) + } } /** diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 16fc0a5a8e53..9560fb23c63a 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -555,11 +555,9 @@ class SparkSession private[sql] ( private[sql] def execute[T]( plan: proto.Plan, - encoder: AgnosticEncoder[T], - observationsOpt: Option[Map[Long, Observation]] = None): SparkResult[T] = { + encoder: AgnosticEncoder[T]): SparkResult[T] = { val value = client.execute(plan) - val result = new SparkResult(value, allocator, encoder, timeZoneId, observationsOpt) - result + new SparkResult(value, allocator, encoder, timeZoneId, Some(this.observationRegistry.toMap)) } private[sql] def execute(f: proto.Relation.Builder => Unit): Unit = { From cc37898062cb21c29c3ce7867b7bca9c6d4c9f22 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Thu, 11 Apr 2024 17:45:51 +0200 Subject: [PATCH 13/20] pyi --- python/pyspark/sql/connect/proto/base_pb2.py | 188 +++++++++--------- python/pyspark/sql/connect/proto/base_pb2.pyi | 93 ++++++++- .../pyspark/sql/connect/proto/catalog_pb2.pyi | 29 +++ .../sql/connect/proto/commands_pb2.pyi | 51 ++++- .../pyspark/sql/connect/proto/common_pb2.pyi | 7 + .../sql/connect/proto/example_plugins_pb2.pyi | 3 + .../sql/connect/proto/expressions_pb2.pyi | 36 +++- .../sql/connect/proto/relations_pb2.pyi | 78 +++++++- .../pyspark/sql/connect/proto/types_pb2.pyi | 27 +++ 9 files changed, 402 insertions(+), 110 deletions(-) diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index b9f88aab3c26..264208e1c98b 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -37,7 +37,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\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.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"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"\xf8\x13\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relationB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xce\r\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevelB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x15\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\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\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\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\x1at\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"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"\x87\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\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\x01R\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)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xab\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx2\xb2\x07\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\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.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"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"\xf8\x13\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relationB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xce\r\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevelB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\x80\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\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\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\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\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"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"\x87\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\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\x01R\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)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xab\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx2\xb2\x07\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\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -120,7 +120,7 @@ _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_start = 5196 _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_end = 5361 _EXECUTEPLANRESPONSE._serialized_start = 5440 - _EXECUTEPLANRESPONSE._serialized_end = 8230 + _EXECUTEPLANRESPONSE._serialized_end = 8256 _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_start = 7030 _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_end = 7101 _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_start = 7103 @@ -133,96 +133,96 @@ _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 7651 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 7653 _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 7741 - _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_start = 7743 - _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_end = 7859 - _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_start = 7861 - _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_end = 7877 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_start = 7880 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_end = 8213 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_start = 8036 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_end = 8213 - _KEYVALUE._serialized_start = 8232 - _KEYVALUE._serialized_end = 8297 - _CONFIGREQUEST._serialized_start = 8300 - _CONFIGREQUEST._serialized_end = 9459 - _CONFIGREQUEST_OPERATION._serialized_start = 8608 - _CONFIGREQUEST_OPERATION._serialized_end = 9106 - _CONFIGREQUEST_SET._serialized_start = 9108 - _CONFIGREQUEST_SET._serialized_end = 9160 - _CONFIGREQUEST_GET._serialized_start = 9162 - _CONFIGREQUEST_GET._serialized_end = 9187 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 9189 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 9252 - _CONFIGREQUEST_GETOPTION._serialized_start = 9254 - _CONFIGREQUEST_GETOPTION._serialized_end = 9285 - _CONFIGREQUEST_GETALL._serialized_start = 9287 - _CONFIGREQUEST_GETALL._serialized_end = 9335 - _CONFIGREQUEST_UNSET._serialized_start = 9337 - _CONFIGREQUEST_UNSET._serialized_end = 9364 - _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 9366 - _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 9400 - _CONFIGRESPONSE._serialized_start = 9462 - _CONFIGRESPONSE._serialized_end = 9637 - _ADDARTIFACTSREQUEST._serialized_start = 9640 - _ADDARTIFACTSREQUEST._serialized_end = 10642 - _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_start = 10115 - _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_end = 10168 - _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_start = 10170 - _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_end = 10281 - _ADDARTIFACTSREQUEST_BATCH._serialized_start = 10283 - _ADDARTIFACTSREQUEST_BATCH._serialized_end = 10376 - _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_start = 10379 - _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_end = 10572 - _ADDARTIFACTSRESPONSE._serialized_start = 10645 - _ADDARTIFACTSRESPONSE._serialized_end = 10917 - _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_start = 10836 - _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_end = 10917 - _ARTIFACTSTATUSESREQUEST._serialized_start = 10920 - _ARTIFACTSTATUSESREQUEST._serialized_end = 11246 - _ARTIFACTSTATUSESRESPONSE._serialized_start = 11249 - _ARTIFACTSTATUSESRESPONSE._serialized_end = 11601 - _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_start = 11444 - _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_end = 11559 - _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_start = 11561 - _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_end = 11601 - _INTERRUPTREQUEST._serialized_start = 11604 - _INTERRUPTREQUEST._serialized_end = 12207 - _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_start = 12007 - _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_end = 12135 - _INTERRUPTRESPONSE._serialized_start = 12210 - _INTERRUPTRESPONSE._serialized_end = 12354 - _REATTACHOPTIONS._serialized_start = 12356 - _REATTACHOPTIONS._serialized_end = 12409 - _REATTACHEXECUTEREQUEST._serialized_start = 12412 - _REATTACHEXECUTEREQUEST._serialized_end = 12818 - _RELEASEEXECUTEREQUEST._serialized_start = 12821 - _RELEASEEXECUTEREQUEST._serialized_end = 13406 - _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_start = 13275 - _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_end = 13287 - _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_start = 13289 - _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_end = 13336 - _RELEASEEXECUTERESPONSE._serialized_start = 13409 - _RELEASEEXECUTERESPONSE._serialized_end = 13574 - _RELEASESESSIONREQUEST._serialized_start = 13577 - _RELEASESESSIONREQUEST._serialized_end = 13748 - _RELEASESESSIONRESPONSE._serialized_start = 13750 - _RELEASESESSIONRESPONSE._serialized_end = 13858 - _FETCHERRORDETAILSREQUEST._serialized_start = 13861 - _FETCHERRORDETAILSREQUEST._serialized_end = 14193 - _FETCHERRORDETAILSRESPONSE._serialized_start = 14196 - _FETCHERRORDETAILSRESPONSE._serialized_end = 15751 - _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_start = 14425 - _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_end = 14599 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_start = 14602 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_end = 14970 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_start = 14933 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_end = 14970 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_start = 14973 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_end = 15382 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_start = 15284 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_end = 15352 - _FETCHERRORDETAILSRESPONSE_ERROR._serialized_start = 15385 - _FETCHERRORDETAILSRESPONSE_ERROR._serialized_end = 15732 - _SPARKCONNECTSERVICE._serialized_start = 15754 - _SPARKCONNECTSERVICE._serialized_end = 16700 + _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_start = 7744 + _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_end = 7885 + _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_start = 7887 + _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_end = 7903 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_start = 7906 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_end = 8239 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_start = 8062 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_end = 8239 + _KEYVALUE._serialized_start = 8258 + _KEYVALUE._serialized_end = 8323 + _CONFIGREQUEST._serialized_start = 8326 + _CONFIGREQUEST._serialized_end = 9485 + _CONFIGREQUEST_OPERATION._serialized_start = 8634 + _CONFIGREQUEST_OPERATION._serialized_end = 9132 + _CONFIGREQUEST_SET._serialized_start = 9134 + _CONFIGREQUEST_SET._serialized_end = 9186 + _CONFIGREQUEST_GET._serialized_start = 9188 + _CONFIGREQUEST_GET._serialized_end = 9213 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 9215 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 9278 + _CONFIGREQUEST_GETOPTION._serialized_start = 9280 + _CONFIGREQUEST_GETOPTION._serialized_end = 9311 + _CONFIGREQUEST_GETALL._serialized_start = 9313 + _CONFIGREQUEST_GETALL._serialized_end = 9361 + _CONFIGREQUEST_UNSET._serialized_start = 9363 + _CONFIGREQUEST_UNSET._serialized_end = 9390 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 9392 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 9426 + _CONFIGRESPONSE._serialized_start = 9488 + _CONFIGRESPONSE._serialized_end = 9663 + _ADDARTIFACTSREQUEST._serialized_start = 9666 + _ADDARTIFACTSREQUEST._serialized_end = 10668 + _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_start = 10141 + _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_end = 10194 + _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_start = 10196 + _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_end = 10307 + _ADDARTIFACTSREQUEST_BATCH._serialized_start = 10309 + _ADDARTIFACTSREQUEST_BATCH._serialized_end = 10402 + _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_start = 10405 + _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_end = 10598 + _ADDARTIFACTSRESPONSE._serialized_start = 10671 + _ADDARTIFACTSRESPONSE._serialized_end = 10943 + _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_start = 10862 + _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_end = 10943 + _ARTIFACTSTATUSESREQUEST._serialized_start = 10946 + _ARTIFACTSTATUSESREQUEST._serialized_end = 11272 + _ARTIFACTSTATUSESRESPONSE._serialized_start = 11275 + _ARTIFACTSTATUSESRESPONSE._serialized_end = 11627 + _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_start = 11470 + _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_end = 11585 + _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_start = 11587 + _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_end = 11627 + _INTERRUPTREQUEST._serialized_start = 11630 + _INTERRUPTREQUEST._serialized_end = 12233 + _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_start = 12033 + _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_end = 12161 + _INTERRUPTRESPONSE._serialized_start = 12236 + _INTERRUPTRESPONSE._serialized_end = 12380 + _REATTACHOPTIONS._serialized_start = 12382 + _REATTACHOPTIONS._serialized_end = 12435 + _REATTACHEXECUTEREQUEST._serialized_start = 12438 + _REATTACHEXECUTEREQUEST._serialized_end = 12844 + _RELEASEEXECUTEREQUEST._serialized_start = 12847 + _RELEASEEXECUTEREQUEST._serialized_end = 13432 + _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_start = 13301 + _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_end = 13313 + _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_start = 13315 + _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_end = 13362 + _RELEASEEXECUTERESPONSE._serialized_start = 13435 + _RELEASEEXECUTERESPONSE._serialized_end = 13600 + _RELEASESESSIONREQUEST._serialized_start = 13603 + _RELEASESESSIONREQUEST._serialized_end = 13774 + _RELEASESESSIONRESPONSE._serialized_start = 13776 + _RELEASESESSIONRESPONSE._serialized_end = 13884 + _FETCHERRORDETAILSREQUEST._serialized_start = 13887 + _FETCHERRORDETAILSREQUEST._serialized_end = 14219 + _FETCHERRORDETAILSRESPONSE._serialized_start = 14222 + _FETCHERRORDETAILSRESPONSE._serialized_end = 15777 + _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_start = 14451 + _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_end = 14625 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_start = 14628 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_end = 14996 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_start = 14959 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_end = 14996 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_start = 14999 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_end = 15408 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_start = 15310 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_end = 15378 + _FETCHERRORDETAILSRESPONSE_ERROR._serialized_start = 15411 + _FETCHERRORDETAILSRESPONSE_ERROR._serialized_end = 15758 + _SPARKCONNECTSERVICE._serialized_start = 15780 + _SPARKCONNECTSERVICE._serialized_end = 16726 # @@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 d22502f8839d..27fc77e77dbe 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -55,6 +55,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Plan(google.protobuf.message.Message): """A [[Plan]] is the structure that carries the runtime information for the execution from the client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference @@ -94,6 +95,7 @@ class Plan(google.protobuf.message.Message): global___Plan = Plan +@typing_extensions.final class UserContext(google.protobuf.message.Message): """User Context is used to refer to one particular user session that is executing queries in the backend. @@ -133,11 +135,13 @@ class UserContext(google.protobuf.message.Message): global___UserContext = UserContext +@typing_extensions.final class AnalyzePlanRequest(google.protobuf.message.Message): """Request to perform plan analyze, optionally to explain the plan.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Schema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -155,6 +159,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class Explain(google.protobuf.message.Message): """Explains the input plan based on a configurable mode.""" @@ -169,7 +174,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): AnalyzePlanRequest.Explain._ExplainMode.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor EXPLAIN_MODE_UNSPECIFIED: AnalyzePlanRequest.Explain._ExplainMode.ValueType # 0 EXPLAIN_MODE_SIMPLE: AnalyzePlanRequest.Explain._ExplainMode.ValueType # 1 @@ -229,6 +234,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): field_name: typing_extensions.Literal["explain_mode", b"explain_mode", "plan", b"plan"], ) -> None: ... + @typing_extensions.final class TreeString(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -261,6 +267,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_level", b"_level"] ) -> typing_extensions.Literal["level"] | None: ... + @typing_extensions.final class IsLocal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -278,6 +285,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class IsStreaming(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -295,6 +303,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class InputFiles(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -312,6 +321,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class SparkVersion(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -319,6 +329,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, ) -> None: ... + @typing_extensions.final class DDLParse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -334,6 +345,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["ddl_string", b"ddl_string"] ) -> None: ... + @typing_extensions.final class SameSemantics(google.protobuf.message.Message): """Returns `true` when the logical query plans are equal and therefore return same results.""" @@ -366,6 +378,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class SemanticHash(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -383,6 +396,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class Persist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -426,6 +440,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_storage_level", b"_storage_level"] ) -> typing_extensions.Literal["storage_level"] | None: ... + @typing_extensions.final class Unpersist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -458,6 +473,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_blocking", b"_blocking"] ) -> typing_extensions.Literal["blocking"] | None: ... + @typing_extensions.final class GetStorageLevel(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -686,6 +702,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): global___AnalyzePlanRequest = AnalyzePlanRequest +@typing_extensions.final class AnalyzePlanResponse(google.protobuf.message.Message): """Response to performing analysis of the query. Contains relevant metadata to be able to reason about the performance. @@ -694,6 +711,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Schema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -712,6 +730,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["schema", b"schema"] ) -> None: ... + @typing_extensions.final class Explain(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -726,6 +745,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["explain_string", b"explain_string"] ) -> None: ... + @typing_extensions.final class TreeString(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -740,6 +760,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["tree_string", b"tree_string"] ) -> None: ... + @typing_extensions.final class IsLocal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -754,6 +775,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["is_local", b"is_local"] ) -> None: ... + @typing_extensions.final class IsStreaming(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -768,6 +790,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["is_streaming", b"is_streaming"] ) -> None: ... + @typing_extensions.final class InputFiles(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -784,6 +807,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["files", b"files"]) -> None: ... + @typing_extensions.final class SparkVersion(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -798,6 +822,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["version", b"version"] ) -> None: ... + @typing_extensions.final class DDLParse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -816,6 +841,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["parsed", b"parsed"] ) -> None: ... + @typing_extensions.final class SameSemantics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -830,6 +856,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... + @typing_extensions.final class SemanticHash(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -844,6 +871,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... + @typing_extensions.final class Persist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -851,6 +879,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, ) -> None: ... + @typing_extensions.final class Unpersist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -858,6 +887,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, ) -> None: ... + @typing_extensions.final class GetStorageLevel(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1035,11 +1065,13 @@ class AnalyzePlanResponse(google.protobuf.message.Message): global___AnalyzePlanResponse = AnalyzePlanResponse +@typing_extensions.final class ExecutePlanRequest(google.protobuf.message.Message): """A request to be executed by the service.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class RequestOption(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1221,6 +1253,7 @@ class ExecutePlanRequest(google.protobuf.message.Message): global___ExecutePlanRequest = ExecutePlanRequest +@typing_extensions.final class ExecutePlanResponse(google.protobuf.message.Message): """The response of a query, can be one or more for each request. Responses belonging to the same input query, carry the same `session_id`. @@ -1229,6 +1262,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class SqlCommandResult(google.protobuf.message.Message): """A SQL command returns an opaque Relation that can be directly used as input for the next call. @@ -1251,6 +1285,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["relation", b"relation"] ) -> None: ... + @typing_extensions.final class ArrowBatch(google.protobuf.message.Message): """Batch results of metrics.""" @@ -1295,12 +1330,15 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_offset", b"_start_offset"] ) -> typing_extensions.Literal["start_offset"] | None: ... + @typing_extensions.final class Metrics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class MetricObject(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ExecutionMetricsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1360,6 +1398,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class MetricValue(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1400,12 +1439,14 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["metrics", b"metrics"] ) -> None: ... + @typing_extensions.final class ObservedMetrics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor NAME_FIELD_NUMBER: builtins.int VALUES_FIELD_NUMBER: builtins.int KEYS_FIELD_NUMBER: builtins.int + PLAN_ID_FIELD_NUMBER: builtins.int name: builtins.str @property def values( @@ -1417,6 +1458,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): def keys( self, ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: ... + plan_id: builtins.int def __init__( self, *, @@ -1426,14 +1468,16 @@ class ExecutePlanResponse(google.protobuf.message.Message): ] | None = ..., keys: collections.abc.Iterable[builtins.str] | None = ..., + plan_id: builtins.int = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "keys", b"keys", "name", b"name", "values", b"values" + "keys", b"keys", "name", b"name", "plan_id", b"plan_id", "values", b"values" ], ) -> None: ... + @typing_extensions.final class ResultComplete(google.protobuf.message.Message): """If present, in a reattachable execution this means that after server sends onComplete, the execution is complete. If the server sends onComplete without sending a ResultComplete, @@ -1446,11 +1490,13 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, ) -> None: ... + @typing_extensions.final class ExecutionProgress(google.protobuf.message.Message): """This message is used to communicate progress about the query progress during the execution.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class StageInfo(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1734,6 +1780,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): global___ExecutePlanResponse = ExecutePlanResponse +@typing_extensions.final class KeyValue(google.protobuf.message.Message): """The key-value pair for the config request and response.""" @@ -1766,11 +1813,13 @@ class KeyValue(google.protobuf.message.Message): global___KeyValue = KeyValue +@typing_extensions.final class ConfigRequest(google.protobuf.message.Message): """Request to update or fetch the configurations.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Operation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1857,6 +1906,7 @@ class ConfigRequest(google.protobuf.message.Message): | None ): ... + @typing_extensions.final class Set(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1873,6 +1923,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... + @typing_extensions.final class Get(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1889,6 +1940,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + @typing_extensions.final class GetWithDefault(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1905,6 +1957,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... + @typing_extensions.final class GetOption(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1921,6 +1974,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + @typing_extensions.final class GetAll(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1942,6 +1996,7 @@ class ConfigRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_prefix", b"_prefix"] ) -> typing_extensions.Literal["prefix"] | None: ... + @typing_extensions.final class Unset(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1958,6 +2013,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + @typing_extensions.final class IsModifiable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2063,6 +2119,7 @@ class ConfigRequest(google.protobuf.message.Message): global___ConfigRequest = ConfigRequest +@typing_extensions.final class ConfigResponse(google.protobuf.message.Message): """Response to the config request. Next ID: 5 @@ -2120,11 +2177,13 @@ class ConfigResponse(google.protobuf.message.Message): global___ConfigResponse = ConfigResponse +@typing_extensions.final class AddArtifactsRequest(google.protobuf.message.Message): """Request to transfer client-local artifacts.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ArtifactChunk(google.protobuf.message.Message): """A chunk of an Artifact.""" @@ -2146,6 +2205,7 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["crc", b"crc", "data", b"data"] ) -> None: ... + @typing_extensions.final class SingleChunkArtifact(google.protobuf.message.Message): """An artifact that is contained in a single `ArtifactChunk`. Generally, this message represents tiny artifacts such as REPL-generated class files. @@ -2179,6 +2239,7 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["data", b"data", "name", b"name"] ) -> None: ... + @typing_extensions.final class Batch(google.protobuf.message.Message): """A number of `SingleChunkArtifact` batched into a single RPC.""" @@ -2201,6 +2262,7 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["artifacts", b"artifacts"] ) -> None: ... + @typing_extensions.final class BeginChunkedArtifact(google.protobuf.message.Message): """Signals the beginning/start of a chunked artifact. A large artifact is transferred through a payload of `BeginChunkedArtifact` followed by a @@ -2370,6 +2432,7 @@ class AddArtifactsRequest(google.protobuf.message.Message): global___AddArtifactsRequest = AddArtifactsRequest +@typing_extensions.final class AddArtifactsResponse(google.protobuf.message.Message): """Response to adding an artifact. Contains relevant metadata to verify successful transfer of artifact(s). @@ -2378,6 +2441,7 @@ class AddArtifactsResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ArtifactSummary(google.protobuf.message.Message): """Metadata of an artifact.""" @@ -2442,6 +2506,7 @@ class AddArtifactsResponse(google.protobuf.message.Message): global___AddArtifactsResponse = AddArtifactsResponse +@typing_extensions.final class ArtifactStatusesRequest(google.protobuf.message.Message): """Request to get current statuses of artifacts at the server side.""" @@ -2542,6 +2607,7 @@ class ArtifactStatusesRequest(google.protobuf.message.Message): global___ArtifactStatusesRequest = ArtifactStatusesRequest +@typing_extensions.final class ArtifactStatusesResponse(google.protobuf.message.Message): """Response to checking artifact statuses. Next ID: 4 @@ -2549,6 +2615,7 @@ class ArtifactStatusesResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class StatusesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2570,6 +2637,7 @@ class ArtifactStatusesResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class ArtifactStatus(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2625,6 +2693,7 @@ class ArtifactStatusesResponse(google.protobuf.message.Message): global___ArtifactStatusesResponse = ArtifactStatusesResponse +@typing_extensions.final class InterruptRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2637,7 +2706,7 @@ class InterruptRequest(google.protobuf.message.Message): InterruptRequest._InterruptType.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor INTERRUPT_TYPE_UNSPECIFIED: InterruptRequest._InterruptType.ValueType # 0 INTERRUPT_TYPE_ALL: InterruptRequest._InterruptType.ValueType # 1 @@ -2766,6 +2835,7 @@ class InterruptRequest(google.protobuf.message.Message): global___InterruptRequest = InterruptRequest +@typing_extensions.final class InterruptResponse(google.protobuf.message.Message): """Next ID: 4""" @@ -2806,6 +2876,7 @@ class InterruptResponse(google.protobuf.message.Message): global___InterruptResponse = InterruptResponse +@typing_extensions.final class ReattachOptions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2831,6 +2902,7 @@ class ReattachOptions(google.protobuf.message.Message): global___ReattachOptions = ReattachOptions +@typing_extensions.final class ReattachExecuteRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2949,9 +3021,11 @@ class ReattachExecuteRequest(google.protobuf.message.Message): global___ReattachExecuteRequest = ReattachExecuteRequest +@typing_extensions.final class ReleaseExecuteRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ReleaseAll(google.protobuf.message.Message): """Release and close operation completely. This will also interrupt the query if it is running execution, and wait for it to be torn down. @@ -2963,6 +3037,7 @@ class ReleaseExecuteRequest(google.protobuf.message.Message): self, ) -> None: ... + @typing_extensions.final class ReleaseUntil(google.protobuf.message.Message): """Release all responses from the operation response stream up to and including the response with the given by response_id. @@ -3099,6 +3174,7 @@ class ReleaseExecuteRequest(google.protobuf.message.Message): global___ReleaseExecuteRequest = ReleaseExecuteRequest +@typing_extensions.final class ReleaseExecuteResponse(google.protobuf.message.Message): """Next ID: 4""" @@ -3150,6 +3226,7 @@ class ReleaseExecuteResponse(google.protobuf.message.Message): global___ReleaseExecuteResponse = ReleaseExecuteResponse +@typing_extensions.final class ReleaseSessionRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3211,6 +3288,7 @@ class ReleaseSessionRequest(google.protobuf.message.Message): global___ReleaseSessionRequest = ReleaseSessionRequest +@typing_extensions.final class ReleaseSessionResponse(google.protobuf.message.Message): """Next ID: 3""" @@ -3239,6 +3317,7 @@ class ReleaseSessionResponse(google.protobuf.message.Message): global___ReleaseSessionResponse = ReleaseSessionResponse +@typing_extensions.final class FetchErrorDetailsRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3327,11 +3406,13 @@ class FetchErrorDetailsRequest(google.protobuf.message.Message): global___FetchErrorDetailsRequest = FetchErrorDetailsRequest +@typing_extensions.final class FetchErrorDetailsResponse(google.protobuf.message.Message): """Next ID: 5""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class StackTraceElement(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3380,6 +3461,7 @@ class FetchErrorDetailsResponse(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_file_name", b"_file_name"] ) -> typing_extensions.Literal["file_name"] | None: ... + @typing_extensions.final class QueryContext(google.protobuf.message.Message): """QueryContext defines the schema for the query context of a SparkThrowable. It helps users understand where the error occurs while executing queries. @@ -3396,7 +3478,7 @@ class FetchErrorDetailsResponse(google.protobuf.message.Message): FetchErrorDetailsResponse.QueryContext._ContextType.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SQL: FetchErrorDetailsResponse.QueryContext._ContextType.ValueType # 0 DATAFRAME: FetchErrorDetailsResponse.QueryContext._ContextType.ValueType # 1 @@ -3470,11 +3552,13 @@ class FetchErrorDetailsResponse(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class SparkThrowable(google.protobuf.message.Message): """SparkThrowable defines the schema for SparkThrowable exceptions.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class MessageParametersEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3564,6 +3648,7 @@ class FetchErrorDetailsResponse(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_sql_state", b"_sql_state"] ) -> typing_extensions.Literal["sql_state"] | None: ... + @typing_extensions.final class Error(google.protobuf.message.Message): """Error defines the schema for the representing exception.""" diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.pyi b/python/pyspark/sql/connect/proto/catalog_pb2.pyi index 2879b44618b5..9ead7e04daf0 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.pyi +++ b/python/pyspark/sql/connect/proto/catalog_pb2.pyi @@ -50,6 +50,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Catalog(google.protobuf.message.Message): """Catalog messages are marked as unstable.""" @@ -317,6 +318,7 @@ class Catalog(google.protobuf.message.Message): global___Catalog = Catalog +@typing_extensions.final class CurrentDatabase(google.protobuf.message.Message): """See `spark.catalog.currentDatabase`""" @@ -328,6 +330,7 @@ class CurrentDatabase(google.protobuf.message.Message): global___CurrentDatabase = CurrentDatabase +@typing_extensions.final class SetCurrentDatabase(google.protobuf.message.Message): """See `spark.catalog.setCurrentDatabase`""" @@ -345,6 +348,7 @@ class SetCurrentDatabase(google.protobuf.message.Message): global___SetCurrentDatabase = SetCurrentDatabase +@typing_extensions.final class ListDatabases(google.protobuf.message.Message): """See `spark.catalog.listDatabases`""" @@ -370,6 +374,7 @@ class ListDatabases(google.protobuf.message.Message): global___ListDatabases = ListDatabases +@typing_extensions.final class ListTables(google.protobuf.message.Message): """See `spark.catalog.listTables`""" @@ -424,6 +429,7 @@ class ListTables(google.protobuf.message.Message): global___ListTables = ListTables +@typing_extensions.final class ListFunctions(google.protobuf.message.Message): """See `spark.catalog.listFunctions`""" @@ -478,6 +484,7 @@ class ListFunctions(google.protobuf.message.Message): global___ListFunctions = ListFunctions +@typing_extensions.final class ListColumns(google.protobuf.message.Message): """See `spark.catalog.listColumns`""" @@ -510,6 +517,7 @@ class ListColumns(google.protobuf.message.Message): global___ListColumns = ListColumns +@typing_extensions.final class GetDatabase(google.protobuf.message.Message): """See `spark.catalog.getDatabase`""" @@ -527,6 +535,7 @@ class GetDatabase(google.protobuf.message.Message): global___GetDatabase = GetDatabase +@typing_extensions.final class GetTable(google.protobuf.message.Message): """See `spark.catalog.getTable`""" @@ -559,6 +568,7 @@ class GetTable(google.protobuf.message.Message): global___GetTable = GetTable +@typing_extensions.final class GetFunction(google.protobuf.message.Message): """See `spark.catalog.getFunction`""" @@ -591,6 +601,7 @@ class GetFunction(google.protobuf.message.Message): global___GetFunction = GetFunction +@typing_extensions.final class DatabaseExists(google.protobuf.message.Message): """See `spark.catalog.databaseExists`""" @@ -608,6 +619,7 @@ class DatabaseExists(google.protobuf.message.Message): global___DatabaseExists = DatabaseExists +@typing_extensions.final class TableExists(google.protobuf.message.Message): """See `spark.catalog.tableExists`""" @@ -640,6 +652,7 @@ class TableExists(google.protobuf.message.Message): global___TableExists = TableExists +@typing_extensions.final class FunctionExists(google.protobuf.message.Message): """See `spark.catalog.functionExists`""" @@ -672,11 +685,13 @@ class FunctionExists(google.protobuf.message.Message): global___FunctionExists = FunctionExists +@typing_extensions.final class CreateExternalTable(google.protobuf.message.Message): """See `spark.catalog.createExternalTable`""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -775,11 +790,13 @@ class CreateExternalTable(google.protobuf.message.Message): global___CreateExternalTable = CreateExternalTable +@typing_extensions.final class CreateTable(google.protobuf.message.Message): """See `spark.catalog.createTable`""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -894,6 +911,7 @@ class CreateTable(google.protobuf.message.Message): global___CreateTable = CreateTable +@typing_extensions.final class DropTempView(google.protobuf.message.Message): """See `spark.catalog.dropTempView`""" @@ -913,6 +931,7 @@ class DropTempView(google.protobuf.message.Message): global___DropTempView = DropTempView +@typing_extensions.final class DropGlobalTempView(google.protobuf.message.Message): """See `spark.catalog.dropGlobalTempView`""" @@ -932,6 +951,7 @@ class DropGlobalTempView(google.protobuf.message.Message): global___DropGlobalTempView = DropGlobalTempView +@typing_extensions.final class RecoverPartitions(google.protobuf.message.Message): """See `spark.catalog.recoverPartitions`""" @@ -951,6 +971,7 @@ class RecoverPartitions(google.protobuf.message.Message): global___RecoverPartitions = RecoverPartitions +@typing_extensions.final class IsCached(google.protobuf.message.Message): """See `spark.catalog.isCached`""" @@ -970,6 +991,7 @@ class IsCached(google.protobuf.message.Message): global___IsCached = IsCached +@typing_extensions.final class CacheTable(google.protobuf.message.Message): """See `spark.catalog.cacheTable`""" @@ -1011,6 +1033,7 @@ class CacheTable(google.protobuf.message.Message): global___CacheTable = CacheTable +@typing_extensions.final class UncacheTable(google.protobuf.message.Message): """See `spark.catalog.uncacheTable`""" @@ -1030,6 +1053,7 @@ class UncacheTable(google.protobuf.message.Message): global___UncacheTable = UncacheTable +@typing_extensions.final class ClearCache(google.protobuf.message.Message): """See `spark.catalog.clearCache`""" @@ -1041,6 +1065,7 @@ class ClearCache(google.protobuf.message.Message): global___ClearCache = ClearCache +@typing_extensions.final class RefreshTable(google.protobuf.message.Message): """See `spark.catalog.refreshTable`""" @@ -1060,6 +1085,7 @@ class RefreshTable(google.protobuf.message.Message): global___RefreshTable = RefreshTable +@typing_extensions.final class RefreshByPath(google.protobuf.message.Message): """See `spark.catalog.refreshByPath`""" @@ -1077,6 +1103,7 @@ class RefreshByPath(google.protobuf.message.Message): global___RefreshByPath = RefreshByPath +@typing_extensions.final class CurrentCatalog(google.protobuf.message.Message): """See `spark.catalog.currentCatalog`""" @@ -1088,6 +1115,7 @@ class CurrentCatalog(google.protobuf.message.Message): global___CurrentCatalog = CurrentCatalog +@typing_extensions.final class SetCurrentCatalog(google.protobuf.message.Message): """See `spark.catalog.setCurrentCatalog`""" @@ -1107,6 +1135,7 @@ class SetCurrentCatalog(google.protobuf.message.Message): global___SetCurrentCatalog = SetCurrentCatalog +@typing_extensions.final class ListCatalogs(google.protobuf.message.Message): """See `spark.catalog.listCatalogs`""" diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index b57a2a6c4d68..bb5690556101 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -60,7 +60,7 @@ class _StreamingQueryEventType: class _StreamingQueryEventTypeEnumTypeWrapper( google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[_StreamingQueryEventType.ValueType], builtins.type, -): # noqa: F821 +): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor QUERY_PROGRESS_UNSPECIFIED: _StreamingQueryEventType.ValueType # 0 QUERY_PROGRESS_EVENT: _StreamingQueryEventType.ValueType # 1 @@ -81,6 +81,7 @@ QUERY_TERMINATED_EVENT: StreamingQueryEventType.ValueType # 2 QUERY_IDLE_EVENT: StreamingQueryEventType.ValueType # 3 global___StreamingQueryEventType = StreamingQueryEventType +@typing_extensions.final class Command(google.protobuf.message.Message): """A [[Command]] is an operation that is executed by the server that does not directly consume or produce a relational result. @@ -255,6 +256,7 @@ class Command(google.protobuf.message.Message): global___Command = Command +@typing_extensions.final class SqlCommand(google.protobuf.message.Message): """A SQL Command is used to trigger the eager evaluation of SQL commands in Spark. @@ -266,6 +268,7 @@ class SqlCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ArgsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -287,6 +290,7 @@ class SqlCommand(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class NamedArgumentsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -386,6 +390,7 @@ class SqlCommand(google.protobuf.message.Message): global___SqlCommand = SqlCommand +@typing_extensions.final class CreateDataFrameViewCommand(google.protobuf.message.Message): """A command that can create DataFrame global temp view or local temp view.""" @@ -428,6 +433,7 @@ class CreateDataFrameViewCommand(google.protobuf.message.Message): global___CreateDataFrameViewCommand = CreateDataFrameViewCommand +@typing_extensions.final class WriteOperation(google.protobuf.message.Message): """As writes are not directly handled during analysis and planning, they are modeled as commands.""" @@ -442,7 +448,7 @@ class WriteOperation(google.protobuf.message.Message): WriteOperation._SaveMode.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SAVE_MODE_UNSPECIFIED: WriteOperation._SaveMode.ValueType # 0 SAVE_MODE_APPEND: WriteOperation._SaveMode.ValueType # 1 @@ -457,6 +463,7 @@ class WriteOperation(google.protobuf.message.Message): SAVE_MODE_ERROR_IF_EXISTS: WriteOperation.SaveMode.ValueType # 3 SAVE_MODE_IGNORE: WriteOperation.SaveMode.ValueType # 4 + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -474,6 +481,7 @@ class WriteOperation(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class SaveTable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -486,7 +494,7 @@ class WriteOperation(google.protobuf.message.Message): WriteOperation.SaveTable._TableSaveMethod.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor TABLE_SAVE_METHOD_UNSPECIFIED: WriteOperation.SaveTable._TableSaveMethod.ValueType # 0 TABLE_SAVE_METHOD_SAVE_AS_TABLE: WriteOperation.SaveTable._TableSaveMethod.ValueType # 1 @@ -516,6 +524,7 @@ class WriteOperation(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class BucketBy(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -655,6 +664,7 @@ class WriteOperation(google.protobuf.message.Message): global___WriteOperation = WriteOperation +@typing_extensions.final class WriteOperationV2(google.protobuf.message.Message): """As writes are not directly handled during analysis and planning, they are modeled as commands.""" @@ -669,7 +679,7 @@ class WriteOperationV2(google.protobuf.message.Message): WriteOperationV2._Mode.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor MODE_UNSPECIFIED: WriteOperationV2._Mode.ValueType # 0 MODE_CREATE: WriteOperationV2._Mode.ValueType # 1 @@ -688,6 +698,7 @@ class WriteOperationV2(google.protobuf.message.Message): MODE_REPLACE: WriteOperationV2.Mode.ValueType # 5 MODE_CREATE_OR_REPLACE: WriteOperationV2.Mode.ValueType # 6 + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -705,6 +716,7 @@ class WriteOperationV2(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class TablePropertiesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -827,6 +839,7 @@ class WriteOperationV2(google.protobuf.message.Message): global___WriteOperationV2 = WriteOperationV2 +@typing_extensions.final class WriteStreamOperationStart(google.protobuf.message.Message): """Starts write stream operation as streaming query. Query ID and Run ID of the streaming query are returned. @@ -834,6 +847,7 @@ class WriteStreamOperationStart(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -990,6 +1004,7 @@ class WriteStreamOperationStart(google.protobuf.message.Message): global___WriteStreamOperationStart = WriteStreamOperationStart +@typing_extensions.final class StreamingForeachFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1033,6 +1048,7 @@ class StreamingForeachFunction(google.protobuf.message.Message): global___StreamingForeachFunction = StreamingForeachFunction +@typing_extensions.final class WriteStreamOperationStartResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1086,6 +1102,7 @@ class WriteStreamOperationStartResult(google.protobuf.message.Message): global___WriteStreamOperationStartResult = WriteStreamOperationStartResult +@typing_extensions.final class StreamingQueryInstanceId(google.protobuf.message.Message): """A tuple that uniquely identifies an instance of streaming query run. It consists of `id` that persists across the streaming runs and `run_id` that changes between each run of the @@ -1118,11 +1135,13 @@ class StreamingQueryInstanceId(google.protobuf.message.Message): global___StreamingQueryInstanceId = StreamingQueryInstanceId +@typing_extensions.final class StreamingQueryCommand(google.protobuf.message.Message): """Commands for a streaming query.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ExplainCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1140,6 +1159,7 @@ class StreamingQueryCommand(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["extended", b"extended"] ) -> None: ... + @typing_extensions.final class AwaitTerminationCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1277,11 +1297,13 @@ class StreamingQueryCommand(google.protobuf.message.Message): global___StreamingQueryCommand = StreamingQueryCommand +@typing_extensions.final class StreamingQueryCommandResult(google.protobuf.message.Message): """Response for commands on a streaming query.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class StatusResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1316,6 +1338,7 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class RecentProgressResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1335,6 +1358,7 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): field_name: typing_extensions.Literal["recent_progress_json", b"recent_progress_json"], ) -> None: ... + @typing_extensions.final class ExplainResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1350,6 +1374,7 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... + @typing_extensions.final class ExceptionResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1419,6 +1444,7 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_stack_trace", b"_stack_trace"] ) -> typing_extensions.Literal["stack_trace"] | None: ... + @typing_extensions.final class AwaitTerminationResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1511,11 +1537,13 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): global___StreamingQueryCommandResult = StreamingQueryCommandResult +@typing_extensions.final class StreamingQueryManagerCommand(google.protobuf.message.Message): """Commands for the streaming query manager.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class AwaitAnyTerminationCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1543,6 +1571,7 @@ class StreamingQueryManagerCommand(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_timeout_ms", b"_timeout_ms"] ) -> typing_extensions.Literal["timeout_ms"] | None: ... + @typing_extensions.final class StreamingQueryListenerCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1693,11 +1722,13 @@ class StreamingQueryManagerCommand(google.protobuf.message.Message): global___StreamingQueryManagerCommand = StreamingQueryManagerCommand +@typing_extensions.final class StreamingQueryManagerCommandResult(google.protobuf.message.Message): """Response for commands on the streaming query manager.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ActiveResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1720,6 +1751,7 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["active_queries", b"active_queries"] ) -> None: ... + @typing_extensions.final class StreamingQueryInstance(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1748,6 +1780,7 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_name", b"_name"] ) -> typing_extensions.Literal["name"] | None: ... + @typing_extensions.final class AwaitAnyTerminationResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1762,6 +1795,7 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["terminated", b"terminated"] ) -> None: ... + @typing_extensions.final class StreamingQueryListenerInstance(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1776,6 +1810,7 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["listener_payload", b"listener_payload"] ) -> None: ... + @typing_extensions.final class ListStreamingQueryListenerResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1888,6 +1923,7 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): global___StreamingQueryManagerCommandResult = StreamingQueryManagerCommandResult +@typing_extensions.final class StreamingQueryListenerBusCommand(google.protobuf.message.Message): """The protocol for client-side StreamingQueryListener. This command will only be set when either the first listener is added to the client, or the last @@ -1939,6 +1975,7 @@ class StreamingQueryListenerBusCommand(google.protobuf.message.Message): global___StreamingQueryListenerBusCommand = StreamingQueryListenerBusCommand +@typing_extensions.final class StreamingQueryListenerEvent(google.protobuf.message.Message): """The protocol for the returned events in the long-running response channel.""" @@ -1965,6 +2002,7 @@ class StreamingQueryListenerEvent(google.protobuf.message.Message): global___StreamingQueryListenerEvent = StreamingQueryListenerEvent +@typing_extensions.final class StreamingQueryListenerEventsResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2012,6 +2050,7 @@ class StreamingQueryListenerEventsResult(google.protobuf.message.Message): global___StreamingQueryListenerEventsResult = StreamingQueryListenerEventsResult +@typing_extensions.final class GetResourcesCommand(google.protobuf.message.Message): """Command to get the output of 'SparkContext.resources'""" @@ -2023,11 +2062,13 @@ class GetResourcesCommand(google.protobuf.message.Message): global___GetResourcesCommand = GetResourcesCommand +@typing_extensions.final class GetResourcesCommandResult(google.protobuf.message.Message): """Response for command 'GetResourcesCommand'.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ResourcesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2070,6 +2111,7 @@ class GetResourcesCommandResult(google.protobuf.message.Message): global___GetResourcesCommandResult = GetResourcesCommandResult +@typing_extensions.final class CreateResourceProfileCommand(google.protobuf.message.Message): """Command to create ResourceProfile""" @@ -2091,6 +2133,7 @@ class CreateResourceProfileCommand(google.protobuf.message.Message): global___CreateResourceProfileCommand = CreateResourceProfileCommand +@typing_extensions.final class CreateResourceProfileCommandResult(google.protobuf.message.Message): """Response for command 'CreateResourceProfileCommand'.""" diff --git a/python/pyspark/sql/connect/proto/common_pb2.pyi b/python/pyspark/sql/connect/proto/common_pb2.pyi index 163781b41998..9845ede2daa2 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.pyi +++ b/python/pyspark/sql/connect/proto/common_pb2.pyi @@ -48,6 +48,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class StorageLevel(google.protobuf.message.Message): """StorageLevel for persisting Datasets/Tables.""" @@ -95,6 +96,7 @@ class StorageLevel(google.protobuf.message.Message): global___StorageLevel = StorageLevel +@typing_extensions.final class ResourceInformation(google.protobuf.message.Message): """ResourceInformation to hold information about a type of Resource. The corresponding class is 'org.apache.spark.resource.ResourceInformation' @@ -123,6 +125,7 @@ class ResourceInformation(google.protobuf.message.Message): global___ResourceInformation = ResourceInformation +@typing_extensions.final class ExecutorResourceRequest(google.protobuf.message.Message): """An executor resource request.""" @@ -189,6 +192,7 @@ class ExecutorResourceRequest(google.protobuf.message.Message): global___ExecutorResourceRequest = ExecutorResourceRequest +@typing_extensions.final class TaskResourceRequest(google.protobuf.message.Message): """A task resource request.""" @@ -217,9 +221,11 @@ class TaskResourceRequest(google.protobuf.message.Message): global___TaskResourceRequest = TaskResourceRequest +@typing_extensions.final class ResourceProfile(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ExecutorResourcesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -241,6 +247,7 @@ class ResourceProfile(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class TaskResourcesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi b/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi index 1be966ff1e40..4a068a7980b3 100644 --- a/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi +++ b/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi @@ -47,6 +47,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class ExamplePluginRelation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -71,6 +72,7 @@ class ExamplePluginRelation(google.protobuf.message.Message): global___ExamplePluginRelation = ExamplePluginRelation +@typing_extensions.final class ExamplePluginExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -95,6 +97,7 @@ class ExamplePluginExpression(google.protobuf.message.Message): global___ExamplePluginExpression = ExamplePluginExpression +@typing_extensions.final class ExamplePluginCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index 183a839da920..20f7f6115063 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -51,6 +51,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Expression(google.protobuf.message.Message): """Expression used to refer to fields, functions and similar. This can be used everywhere expressions in SQL appear. @@ -58,11 +59,13 @@ class Expression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Window(google.protobuf.message.Message): """Expression for the OVER clause or WINDOW clause.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class WindowFrame(google.protobuf.message.Message): """The window frame""" @@ -77,7 +80,7 @@ class Expression(google.protobuf.message.Message): Expression.Window.WindowFrame._FrameType.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor FRAME_TYPE_UNDEFINED: Expression.Window.WindowFrame._FrameType.ValueType # 0 FRAME_TYPE_ROW: Expression.Window.WindowFrame._FrameType.ValueType # 1 @@ -96,6 +99,7 @@ class Expression(google.protobuf.message.Message): All rows having the same 'ORDER BY' ordering are considered as peers. """ + @typing_extensions.final class FrameBoundary(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -232,6 +236,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class SortOrder(google.protobuf.message.Message): """SortOrder is used to specify the data ordering, it is normally used in Sort and Window. It is an unevaluable expression and cannot be evaluated, so can not be used in Projection. @@ -248,7 +253,7 @@ class Expression(google.protobuf.message.Message): Expression.SortOrder._SortDirection.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SORT_DIRECTION_UNSPECIFIED: Expression.SortOrder._SortDirection.ValueType # 0 SORT_DIRECTION_ASCENDING: Expression.SortOrder._SortDirection.ValueType # 1 @@ -268,7 +273,7 @@ class Expression(google.protobuf.message.Message): Expression.SortOrder._NullOrdering.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SORT_NULLS_UNSPECIFIED: Expression.SortOrder._NullOrdering.ValueType # 0 SORT_NULLS_FIRST: Expression.SortOrder._NullOrdering.ValueType # 1 @@ -306,6 +311,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Cast(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -318,7 +324,7 @@ class Expression(google.protobuf.message.Message): Expression.Cast._EvalMode.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor EVAL_MODE_UNSPECIFIED: Expression.Cast._EvalMode.ValueType # 0 EVAL_MODE_LEGACY: Expression.Cast._EvalMode.ValueType # 1 @@ -384,9 +390,11 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["cast_to_type", b"cast_to_type"] ) -> typing_extensions.Literal["type", "type_str"] | None: ... + @typing_extensions.final class Literal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Decimal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -445,6 +453,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_scale", b"_scale"] ) -> typing_extensions.Literal["scale"] | None: ... + @typing_extensions.final class CalendarInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -468,6 +477,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Array(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -497,6 +507,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Map(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -548,6 +559,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Struct(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -772,6 +784,7 @@ class Expression(google.protobuf.message.Message): | None ): ... + @typing_extensions.final class UnresolvedAttribute(google.protobuf.message.Message): """An unresolved attribute that is not explicitly bound to a specific column, but the column is resolved during analysis by name. @@ -835,6 +848,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_plan_id", b"_plan_id"] ) -> typing_extensions.Literal["plan_id"] | None: ... + @typing_extensions.final class UnresolvedFunction(google.protobuf.message.Message): """An unresolved function is not explicitly bound to one explicit function, but the function is resolved during analysis following Sparks name resolution rules. @@ -885,6 +899,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class ExpressionString(google.protobuf.message.Message): """Expression as string.""" @@ -902,6 +917,7 @@ class Expression(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["expression", b"expression"] ) -> None: ... + @typing_extensions.final class UnresolvedStar(google.protobuf.message.Message): """UnresolvedStar is used to expand all the fields of a relation or struct.""" @@ -958,6 +974,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_unparsed_target", b"_unparsed_target"] ) -> typing_extensions.Literal["unparsed_target"] | None: ... + @typing_extensions.final class UnresolvedRegex(google.protobuf.message.Message): """Represents all of the input attributes to a given relational operator, for example in "SELECT `(id)?+.+` FROM ...". @@ -991,6 +1008,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_plan_id", b"_plan_id"] ) -> typing_extensions.Literal["plan_id"] | None: ... + @typing_extensions.final class UnresolvedExtractValue(google.protobuf.message.Message): """Extracts a value or values from an Expression""" @@ -1023,6 +1041,7 @@ class Expression(google.protobuf.message.Message): field_name: typing_extensions.Literal["child", b"child", "extraction", b"extraction"], ) -> None: ... + @typing_extensions.final class UpdateFields(google.protobuf.message.Message): """Add, replace or drop a field of `StructType` expression by name.""" @@ -1067,6 +1086,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Alias(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1109,6 +1129,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_metadata", b"_metadata"] ) -> typing_extensions.Literal["metadata"] | None: ... + @typing_extensions.final class LambdaFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1145,6 +1166,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class UnresolvedNamedLambdaVariable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1359,6 +1381,7 @@ class Expression(google.protobuf.message.Message): global___Expression = Expression +@typing_extensions.final class CommonInlineUserDefinedFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1431,6 +1454,7 @@ class CommonInlineUserDefinedFunction(google.protobuf.message.Message): global___CommonInlineUserDefinedFunction = CommonInlineUserDefinedFunction +@typing_extensions.final class PythonUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1474,6 +1498,7 @@ class PythonUDF(google.protobuf.message.Message): global___PythonUDF = PythonUDF +@typing_extensions.final class ScalarScalaUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1523,6 +1548,7 @@ class ScalarScalaUDF(google.protobuf.message.Message): global___ScalarScalaUDF = ScalarScalaUDF +@typing_extensions.final class JavaUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1568,6 +1594,7 @@ class JavaUDF(google.protobuf.message.Message): global___JavaUDF = JavaUDF +@typing_extensions.final class CallFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1595,6 +1622,7 @@ class CallFunction(google.protobuf.message.Message): global___CallFunction = CallFunction +@typing_extensions.final class NamedArgumentExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index db9609eebb85..9df01405453d 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -53,6 +53,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Relation(google.protobuf.message.Message): """The main [[Relation]] type. Fundamentally, a relation is a typed container that has exactly one explicit relation type set. @@ -587,6 +588,7 @@ class Relation(google.protobuf.message.Message): global___Relation = Relation +@typing_extensions.final class Unknown(google.protobuf.message.Message): """Used for testing purposes only.""" @@ -598,6 +600,7 @@ class Unknown(google.protobuf.message.Message): global___Unknown = Unknown +@typing_extensions.final class RelationCommon(google.protobuf.message.Message): """Common metadata of all relations.""" @@ -630,11 +633,13 @@ class RelationCommon(google.protobuf.message.Message): global___RelationCommon = RelationCommon +@typing_extensions.final class SQL(google.protobuf.message.Message): """Relation that uses a SQL query to generate the output.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ArgsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -656,6 +661,7 @@ class SQL(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class NamedArgumentsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -755,6 +761,7 @@ class SQL(google.protobuf.message.Message): global___SQL = SQL +@typing_extensions.final class Read(google.protobuf.message.Message): """Relation that reads from a file / table or other data source. Does not have additional inputs. @@ -762,9 +769,11 @@ class Read(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class NamedTable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -804,9 +813,11 @@ class Read(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class DataSource(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -951,6 +962,7 @@ class Read(google.protobuf.message.Message): global___Read = Read +@typing_extensions.final class Project(google.protobuf.message.Message): """Projection of a bag of expressions for a given input relation. @@ -992,6 +1004,7 @@ class Project(google.protobuf.message.Message): global___Project = Project +@typing_extensions.final class Filter(google.protobuf.message.Message): """Relation that applies a boolean expression `condition` on each row of `input` to produce the output result. @@ -1022,6 +1035,7 @@ class Filter(google.protobuf.message.Message): global___Filter = Filter +@typing_extensions.final class Join(google.protobuf.message.Message): """Relation of type [[Join]]. @@ -1037,7 +1051,7 @@ class Join(google.protobuf.message.Message): class _JoinTypeEnumTypeWrapper( google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[Join._JoinType.ValueType], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor JOIN_TYPE_UNSPECIFIED: Join._JoinType.ValueType # 0 JOIN_TYPE_INNER: Join._JoinType.ValueType # 1 @@ -1058,6 +1072,7 @@ class Join(google.protobuf.message.Message): JOIN_TYPE_LEFT_SEMI: Join.JoinType.ValueType # 6 JOIN_TYPE_CROSS: Join.JoinType.ValueType # 7 + @typing_extensions.final class JoinDataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1163,6 +1178,7 @@ class Join(google.protobuf.message.Message): global___Join = Join +@typing_extensions.final class SetOperation(google.protobuf.message.Message): """Relation of type [[SetOperation]]""" @@ -1177,7 +1193,7 @@ class SetOperation(google.protobuf.message.Message): SetOperation._SetOpType.ValueType ], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SET_OP_TYPE_UNSPECIFIED: SetOperation._SetOpType.ValueType # 0 SET_OP_TYPE_INTERSECT: SetOperation._SetOpType.ValueType # 1 @@ -1290,6 +1306,7 @@ class SetOperation(google.protobuf.message.Message): global___SetOperation = SetOperation +@typing_extensions.final class Limit(google.protobuf.message.Message): """Relation of type [[Limit]] that is used to `limit` rows from the input relation.""" @@ -1317,6 +1334,7 @@ class Limit(google.protobuf.message.Message): global___Limit = Limit +@typing_extensions.final class Offset(google.protobuf.message.Message): """Relation of type [[Offset]] that is used to read rows staring from the `offset` on the input relation. @@ -1346,6 +1364,7 @@ class Offset(google.protobuf.message.Message): global___Offset = Offset +@typing_extensions.final class Tail(google.protobuf.message.Message): """Relation of type [[Tail]] that is used to fetch `limit` rows from the last of the input relation.""" @@ -1373,6 +1392,7 @@ class Tail(google.protobuf.message.Message): global___Tail = Tail +@typing_extensions.final class Aggregate(google.protobuf.message.Message): """Relation of type [[Aggregate]].""" @@ -1385,7 +1405,7 @@ class Aggregate(google.protobuf.message.Message): class _GroupTypeEnumTypeWrapper( google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[Aggregate._GroupType.ValueType], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor GROUP_TYPE_UNSPECIFIED: Aggregate._GroupType.ValueType # 0 GROUP_TYPE_GROUPBY: Aggregate._GroupType.ValueType # 1 @@ -1402,6 +1422,7 @@ class Aggregate(google.protobuf.message.Message): GROUP_TYPE_PIVOT: Aggregate.GroupType.ValueType # 4 GROUP_TYPE_GROUPING_SETS: Aggregate.GroupType.ValueType # 5 + @typing_extensions.final class Pivot(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1437,6 +1458,7 @@ class Aggregate(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["col", b"col", "values", b"values"] ) -> None: ... + @typing_extensions.final class GroupingSets(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1534,6 +1556,7 @@ class Aggregate(google.protobuf.message.Message): global___Aggregate = Aggregate +@typing_extensions.final class Sort(google.protobuf.message.Message): """Relation of type [[Sort]].""" @@ -1589,6 +1612,7 @@ class Sort(google.protobuf.message.Message): global___Sort = Sort +@typing_extensions.final class Drop(google.protobuf.message.Message): """Drop specified columns.""" @@ -1632,6 +1656,7 @@ class Drop(google.protobuf.message.Message): global___Drop = Drop +@typing_extensions.final class Deduplicate(google.protobuf.message.Message): """Relation of type [[Deduplicate]] which have duplicate rows removed, could consider either only the subset of columns or all the columns. @@ -1713,6 +1738,7 @@ class Deduplicate(google.protobuf.message.Message): global___Deduplicate = Deduplicate +@typing_extensions.final class LocalRelation(google.protobuf.message.Message): """A relation that does not need to be qualified by name.""" @@ -1760,6 +1786,7 @@ class LocalRelation(google.protobuf.message.Message): global___LocalRelation = LocalRelation +@typing_extensions.final class CachedLocalRelation(google.protobuf.message.Message): """A local relation that has been cached already.""" @@ -1777,6 +1804,7 @@ class CachedLocalRelation(google.protobuf.message.Message): global___CachedLocalRelation = CachedLocalRelation +@typing_extensions.final class CachedRemoteRelation(google.protobuf.message.Message): """Represents a remote relation that has been cached on server.""" @@ -1796,6 +1824,7 @@ class CachedRemoteRelation(google.protobuf.message.Message): global___CachedRemoteRelation = CachedRemoteRelation +@typing_extensions.final class Sample(google.protobuf.message.Message): """Relation of type [[Sample]] that samples a fraction of the dataset.""" @@ -1880,6 +1909,7 @@ class Sample(google.protobuf.message.Message): global___Sample = Sample +@typing_extensions.final class Range(google.protobuf.message.Message): """Relation of type [[Range]] that generates a sequence of integers.""" @@ -1948,6 +1978,7 @@ class Range(google.protobuf.message.Message): global___Range = Range +@typing_extensions.final class SubqueryAlias(google.protobuf.message.Message): """Relation alias.""" @@ -1985,6 +2016,7 @@ class SubqueryAlias(google.protobuf.message.Message): global___SubqueryAlias = SubqueryAlias +@typing_extensions.final class Repartition(google.protobuf.message.Message): """Relation repartition.""" @@ -2032,6 +2064,7 @@ class Repartition(google.protobuf.message.Message): global___Repartition = Repartition +@typing_extensions.final class ShowString(google.protobuf.message.Message): """Compose the string representing rows for output. It will invoke 'Dataset.showString' to compute the results. @@ -2081,6 +2114,7 @@ class ShowString(google.protobuf.message.Message): global___ShowString = ShowString +@typing_extensions.final class HtmlString(google.protobuf.message.Message): """Compose the string representing rows for output. It will invoke 'Dataset.htmlString' to compute the results. @@ -2119,6 +2153,7 @@ class HtmlString(google.protobuf.message.Message): global___HtmlString = HtmlString +@typing_extensions.final class StatSummary(google.protobuf.message.Message): """Computes specified statistics for numeric and string columns. It will invoke 'Dataset.summary' (same as 'StatFunctions.summary') @@ -2166,6 +2201,7 @@ class StatSummary(google.protobuf.message.Message): global___StatSummary = StatSummary +@typing_extensions.final class StatDescribe(google.protobuf.message.Message): """Computes basic statistics for numeric and string columns, including count, mean, stddev, min, and max. If no columns are given, this function computes statistics for all numerical or @@ -2199,6 +2235,7 @@ class StatDescribe(google.protobuf.message.Message): global___StatDescribe = StatDescribe +@typing_extensions.final class StatCrosstab(google.protobuf.message.Message): """Computes a pair-wise frequency table of the given columns. Also known as a contingency table. It will invoke 'Dataset.stat.crosstab' (same as 'StatFunctions.crossTabulate') @@ -2240,6 +2277,7 @@ class StatCrosstab(google.protobuf.message.Message): global___StatCrosstab = StatCrosstab +@typing_extensions.final class StatCov(google.protobuf.message.Message): """Calculate the sample covariance of two numerical columns of a DataFrame. It will invoke 'Dataset.stat.cov' (same as 'StatFunctions.calculateCov') to compute the results. @@ -2274,6 +2312,7 @@ class StatCov(google.protobuf.message.Message): global___StatCov = StatCov +@typing_extensions.final class StatCorr(google.protobuf.message.Message): """Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson Correlation Coefficient. It will invoke 'Dataset.stat.corr' (same as @@ -2333,6 +2372,7 @@ class StatCorr(google.protobuf.message.Message): global___StatCorr = StatCorr +@typing_extensions.final class StatApproxQuantile(google.protobuf.message.Message): """Calculates the approximate quantiles of numerical columns of a DataFrame. It will invoke 'Dataset.stat.approxQuantile' (same as 'StatFunctions.approxQuantile') @@ -2395,6 +2435,7 @@ class StatApproxQuantile(google.protobuf.message.Message): global___StatApproxQuantile = StatApproxQuantile +@typing_extensions.final class StatFreqItems(google.protobuf.message.Message): """Finding frequent items for columns, possibly with false positives. It will invoke 'Dataset.stat.freqItems' (same as 'StatFunctions.freqItems') @@ -2443,6 +2484,7 @@ class StatFreqItems(google.protobuf.message.Message): global___StatFreqItems = StatFreqItems +@typing_extensions.final class StatSampleBy(google.protobuf.message.Message): """Returns a stratified sample without replacement based on the fraction given on each stratum. @@ -2452,6 +2494,7 @@ class StatSampleBy(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Fraction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2533,6 +2576,7 @@ class StatSampleBy(google.protobuf.message.Message): global___StatSampleBy = StatSampleBy +@typing_extensions.final class NAFill(google.protobuf.message.Message): """Replaces null values. It will invoke 'Dataset.na.fill' (same as 'DataFrameNaFunctions.fill') to compute the results. @@ -2591,6 +2635,7 @@ class NAFill(google.protobuf.message.Message): global___NAFill = NAFill +@typing_extensions.final class NADrop(google.protobuf.message.Message): """Drop rows containing null values. It will invoke 'Dataset.na.drop' (same as 'DataFrameNaFunctions.drop') to compute the results. @@ -2659,6 +2704,7 @@ class NADrop(google.protobuf.message.Message): global___NADrop = NADrop +@typing_extensions.final class NAReplace(google.protobuf.message.Message): """Replaces old values with the corresponding values. It will invoke 'Dataset.na.replace' (same as 'DataFrameNaFunctions.replace') @@ -2667,6 +2713,7 @@ class NAReplace(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Replacement(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2743,6 +2790,7 @@ class NAReplace(google.protobuf.message.Message): global___NAReplace = NAReplace +@typing_extensions.final class ToDF(google.protobuf.message.Message): """Rename columns on the input relation by the same length of names.""" @@ -2778,11 +2826,13 @@ class ToDF(google.protobuf.message.Message): global___ToDF = ToDF +@typing_extensions.final class WithColumnsRenamed(google.protobuf.message.Message): """Rename columns on the input relation by a map with name to name mapping.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class RenameColumnsMapEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2800,6 +2850,7 @@ class WithColumnsRenamed(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class Rename(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2864,6 +2915,7 @@ class WithColumnsRenamed(google.protobuf.message.Message): global___WithColumnsRenamed = WithColumnsRenamed +@typing_extensions.final class WithColumns(google.protobuf.message.Message): """Adding columns or replacing the existing columns that have the same names.""" @@ -2908,6 +2960,7 @@ class WithColumns(google.protobuf.message.Message): global___WithColumns = WithColumns +@typing_extensions.final class WithWatermark(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2940,6 +2993,7 @@ class WithWatermark(google.protobuf.message.Message): global___WithWatermark = WithWatermark +@typing_extensions.final class Hint(google.protobuf.message.Message): """Specify a hint over a relation. Hint should have a name and optional parameters.""" @@ -2985,11 +3039,13 @@ class Hint(google.protobuf.message.Message): global___Hint = Hint +@typing_extensions.final class Unpivot(google.protobuf.message.Message): """Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Values(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3071,6 +3127,7 @@ class Unpivot(google.protobuf.message.Message): global___Unpivot = Unpivot +@typing_extensions.final class ToSchema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3100,6 +3157,7 @@ class ToSchema(google.protobuf.message.Message): global___ToSchema = ToSchema +@typing_extensions.final class RepartitionByExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3158,6 +3216,7 @@ class RepartitionByExpression(google.protobuf.message.Message): global___RepartitionByExpression = RepartitionByExpression +@typing_extensions.final class MapPartitions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3229,6 +3288,7 @@ class MapPartitions(google.protobuf.message.Message): global___MapPartitions = MapPartitions +@typing_extensions.final class GroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3372,6 +3432,7 @@ class GroupMap(google.protobuf.message.Message): global___GroupMap = GroupMap +@typing_extensions.final class CoGroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3471,6 +3532,7 @@ class CoGroupMap(google.protobuf.message.Message): global___CoGroupMap = CoGroupMap +@typing_extensions.final class ApplyInPandasWithState(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3542,6 +3604,7 @@ class ApplyInPandasWithState(google.protobuf.message.Message): global___ApplyInPandasWithState = ApplyInPandasWithState +@typing_extensions.final class CommonInlineUserDefinedTableFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3598,6 +3661,7 @@ class CommonInlineUserDefinedTableFunction(google.protobuf.message.Message): global___CommonInlineUserDefinedTableFunction = CommonInlineUserDefinedTableFunction +@typing_extensions.final class PythonUDTF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3649,6 +3713,7 @@ class PythonUDTF(google.protobuf.message.Message): global___PythonUDTF = PythonUDTF +@typing_extensions.final class CommonInlineUserDefinedDataSource(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3687,6 +3752,7 @@ class CommonInlineUserDefinedDataSource(google.protobuf.message.Message): global___CommonInlineUserDefinedDataSource = CommonInlineUserDefinedDataSource +@typing_extensions.final class PythonDataSource(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3709,6 +3775,7 @@ class PythonDataSource(google.protobuf.message.Message): global___PythonDataSource = PythonDataSource +@typing_extensions.final class CollectMetrics(google.protobuf.message.Message): """Collect arbitrary (named) metrics from a dataset.""" @@ -3749,6 +3816,7 @@ class CollectMetrics(google.protobuf.message.Message): global___CollectMetrics = CollectMetrics +@typing_extensions.final class Parse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3759,7 +3827,7 @@ class Parse(google.protobuf.message.Message): class _ParseFormatEnumTypeWrapper( google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[Parse._ParseFormat.ValueType], builtins.type, - ): # noqa: F821 + ): DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor PARSE_FORMAT_UNSPECIFIED: Parse._ParseFormat.ValueType # 0 PARSE_FORMAT_CSV: Parse._ParseFormat.ValueType # 1 @@ -3770,6 +3838,7 @@ class Parse(google.protobuf.message.Message): PARSE_FORMAT_CSV: Parse.ParseFormat.ValueType # 1 PARSE_FORMAT_JSON: Parse.ParseFormat.ValueType # 2 + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3837,6 +3906,7 @@ class Parse(google.protobuf.message.Message): global___Parse = Parse +@typing_extensions.final class AsOfJoin(google.protobuf.message.Message): """Relation of type [[AsOfJoin]]. diff --git a/python/pyspark/sql/connect/proto/types_pb2.pyi b/python/pyspark/sql/connect/proto/types_pb2.pyi index e6b34d3485c2..8276c9d24918 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.pyi +++ b/python/pyspark/sql/connect/proto/types_pb2.pyi @@ -48,6 +48,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class DataType(google.protobuf.message.Message): """This message describes the logical [[DataType]] of something. It does not carry the value itself but only describes it. @@ -55,6 +56,7 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Boolean(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -72,6 +74,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Byte(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -89,6 +92,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Short(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -106,6 +110,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Integer(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -123,6 +128,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Long(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -140,6 +146,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Float(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -157,6 +164,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Double(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -174,6 +182,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class String(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -197,6 +206,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Binary(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -214,6 +224,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class NULL(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -231,6 +242,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Timestamp(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -248,6 +260,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Date(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -265,6 +278,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class TimestampNTZ(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -282,6 +296,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class CalendarInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -299,6 +314,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class YearMonthInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -352,6 +368,7 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_field", b"_start_field"] ) -> typing_extensions.Literal["start_field"] | None: ... + @typing_extensions.final class DayTimeInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -405,6 +422,7 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_field", b"_start_field"] ) -> typing_extensions.Literal["start_field"] | None: ... + @typing_extensions.final class Char(google.protobuf.message.Message): """Start compound types.""" @@ -427,6 +445,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class VarChar(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -447,6 +466,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Decimal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -500,6 +520,7 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_scale", b"_scale"] ) -> typing_extensions.Literal["scale"] | None: ... + @typing_extensions.final class StructField(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -545,6 +566,7 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_metadata", b"_metadata"] ) -> typing_extensions.Literal["metadata"] | None: ... + @typing_extensions.final class Struct(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -570,6 +592,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Array(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -602,6 +625,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Map(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -643,6 +667,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Variant(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -660,6 +685,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class UDT(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -739,6 +765,7 @@ class DataType(google.protobuf.message.Message): ], ) -> typing_extensions.Literal["serialized_python_class"] | None: ... + @typing_extensions.final class Unparsed(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor From 4c1dda7a807b415a3e0be9a72ef70f296ee275d1 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Thu, 11 Apr 2024 17:52:17 +0200 Subject: [PATCH 14/20] fmt --- .../scala/org/apache/spark/sql/SparkSession.scala | 4 +--- .../execution/SparkConnectPlanExecution.scala | 13 +++++++------ 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 9560fb23c63a..1904cbd219db 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -553,9 +553,7 @@ class SparkSession private[sql] ( private[sql] def timeZoneId: String = conf.get(SqlApiConf.SESSION_LOCAL_TIMEZONE_KEY) - private[sql] def execute[T]( - plan: proto.Plan, - encoder: AgnosticEncoder[T]): SparkResult[T] = { + private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { val value = client.execute(plan) new SparkResult(value, allocator, encoder, timeZoneId, Some(this.observationRegistry.toMap)) } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala index 34a99cfb6ebd..ebea03e6c67a 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala @@ -254,12 +254,13 @@ private[execution] class SparkConnectPlanExecution(executeHolder: ExecuteHolder) } if (observedMetrics.nonEmpty) { val planId = executeHolder.request.getPlan.getRoot.getCommon.getPlanId - Some(SparkConnectPlanExecution - .createObservedMetricsResponse( - sessionId, - sessionHolder.serverSessionId, - planId, - observedMetrics)) + Some( + SparkConnectPlanExecution + .createObservedMetricsResponse( + sessionId, + sessionHolder.serverSessionId, + planId, + observedMetrics)) } else None } } From ef8a6d6085715209063f70cb80ae2fa52e63c2eb Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Thu, 11 Apr 2024 17:58:16 +0200 Subject: [PATCH 15/20] Revert "pyi" This reverts commit cc37898062cb21c29c3ce7867b7bca9c6d4c9f22. --- python/pyspark/sql/connect/proto/base_pb2.pyi | 88 +------------------ .../pyspark/sql/connect/proto/catalog_pb2.pyi | 29 ------ .../sql/connect/proto/commands_pb2.pyi | 51 +---------- .../pyspark/sql/connect/proto/common_pb2.pyi | 7 -- .../sql/connect/proto/example_plugins_pb2.pyi | 3 - .../sql/connect/proto/expressions_pb2.pyi | 36 +------- .../sql/connect/proto/relations_pb2.pyi | 78 +--------------- .../pyspark/sql/connect/proto/types_pb2.pyi | 27 ------ 8 files changed, 15 insertions(+), 304 deletions(-) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 27fc77e77dbe..b76f2a7f4de3 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -55,7 +55,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Plan(google.protobuf.message.Message): """A [[Plan]] is the structure that carries the runtime information for the execution from the client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference @@ -95,7 +94,6 @@ class Plan(google.protobuf.message.Message): global___Plan = Plan -@typing_extensions.final class UserContext(google.protobuf.message.Message): """User Context is used to refer to one particular user session that is executing queries in the backend. @@ -135,13 +133,11 @@ class UserContext(google.protobuf.message.Message): global___UserContext = UserContext -@typing_extensions.final class AnalyzePlanRequest(google.protobuf.message.Message): """Request to perform plan analyze, optionally to explain the plan.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Schema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -159,7 +155,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class Explain(google.protobuf.message.Message): """Explains the input plan based on a configurable mode.""" @@ -174,7 +169,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): AnalyzePlanRequest.Explain._ExplainMode.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor EXPLAIN_MODE_UNSPECIFIED: AnalyzePlanRequest.Explain._ExplainMode.ValueType # 0 EXPLAIN_MODE_SIMPLE: AnalyzePlanRequest.Explain._ExplainMode.ValueType # 1 @@ -234,7 +229,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): field_name: typing_extensions.Literal["explain_mode", b"explain_mode", "plan", b"plan"], ) -> None: ... - @typing_extensions.final class TreeString(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -267,7 +261,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_level", b"_level"] ) -> typing_extensions.Literal["level"] | None: ... - @typing_extensions.final class IsLocal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -285,7 +278,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class IsStreaming(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -303,7 +295,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class InputFiles(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -321,7 +312,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class SparkVersion(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -329,7 +319,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, ) -> None: ... - @typing_extensions.final class DDLParse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -345,7 +334,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["ddl_string", b"ddl_string"] ) -> None: ... - @typing_extensions.final class SameSemantics(google.protobuf.message.Message): """Returns `true` when the logical query plans are equal and therefore return same results.""" @@ -378,7 +366,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class SemanticHash(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -396,7 +383,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class Persist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -440,7 +426,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_storage_level", b"_storage_level"] ) -> typing_extensions.Literal["storage_level"] | None: ... - @typing_extensions.final class Unpersist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -473,7 +458,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_blocking", b"_blocking"] ) -> typing_extensions.Literal["blocking"] | None: ... - @typing_extensions.final class GetStorageLevel(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -702,7 +686,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): global___AnalyzePlanRequest = AnalyzePlanRequest -@typing_extensions.final class AnalyzePlanResponse(google.protobuf.message.Message): """Response to performing analysis of the query. Contains relevant metadata to be able to reason about the performance. @@ -711,7 +694,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Schema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -730,7 +712,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["schema", b"schema"] ) -> None: ... - @typing_extensions.final class Explain(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -745,7 +726,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["explain_string", b"explain_string"] ) -> None: ... - @typing_extensions.final class TreeString(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -760,7 +740,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["tree_string", b"tree_string"] ) -> None: ... - @typing_extensions.final class IsLocal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -775,7 +754,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["is_local", b"is_local"] ) -> None: ... - @typing_extensions.final class IsStreaming(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -790,7 +768,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["is_streaming", b"is_streaming"] ) -> None: ... - @typing_extensions.final class InputFiles(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -807,7 +784,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["files", b"files"]) -> None: ... - @typing_extensions.final class SparkVersion(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -822,7 +798,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["version", b"version"] ) -> None: ... - @typing_extensions.final class DDLParse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -841,7 +816,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["parsed", b"parsed"] ) -> None: ... - @typing_extensions.final class SameSemantics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -856,7 +830,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... - @typing_extensions.final class SemanticHash(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -871,7 +844,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... - @typing_extensions.final class Persist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -879,7 +851,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, ) -> None: ... - @typing_extensions.final class Unpersist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -887,7 +858,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, ) -> None: ... - @typing_extensions.final class GetStorageLevel(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1065,13 +1035,11 @@ class AnalyzePlanResponse(google.protobuf.message.Message): global___AnalyzePlanResponse = AnalyzePlanResponse -@typing_extensions.final class ExecutePlanRequest(google.protobuf.message.Message): """A request to be executed by the service.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class RequestOption(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1253,7 +1221,6 @@ class ExecutePlanRequest(google.protobuf.message.Message): global___ExecutePlanRequest = ExecutePlanRequest -@typing_extensions.final class ExecutePlanResponse(google.protobuf.message.Message): """The response of a query, can be one or more for each request. Responses belonging to the same input query, carry the same `session_id`. @@ -1262,7 +1229,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class SqlCommandResult(google.protobuf.message.Message): """A SQL command returns an opaque Relation that can be directly used as input for the next call. @@ -1285,7 +1251,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["relation", b"relation"] ) -> None: ... - @typing_extensions.final class ArrowBatch(google.protobuf.message.Message): """Batch results of metrics.""" @@ -1330,15 +1295,12 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_offset", b"_start_offset"] ) -> typing_extensions.Literal["start_offset"] | None: ... - @typing_extensions.final class Metrics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class MetricObject(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ExecutionMetricsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1398,7 +1360,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class MetricValue(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1439,7 +1400,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["metrics", b"metrics"] ) -> None: ... - @typing_extensions.final class ObservedMetrics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1477,7 +1437,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class ResultComplete(google.protobuf.message.Message): """If present, in a reattachable execution this means that after server sends onComplete, the execution is complete. If the server sends onComplete without sending a ResultComplete, @@ -1490,13 +1449,11 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, ) -> None: ... - @typing_extensions.final class ExecutionProgress(google.protobuf.message.Message): """This message is used to communicate progress about the query progress during the execution.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class StageInfo(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1780,7 +1737,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): global___ExecutePlanResponse = ExecutePlanResponse -@typing_extensions.final class KeyValue(google.protobuf.message.Message): """The key-value pair for the config request and response.""" @@ -1813,13 +1769,11 @@ class KeyValue(google.protobuf.message.Message): global___KeyValue = KeyValue -@typing_extensions.final class ConfigRequest(google.protobuf.message.Message): """Request to update or fetch the configurations.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Operation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1906,7 +1860,6 @@ class ConfigRequest(google.protobuf.message.Message): | None ): ... - @typing_extensions.final class Set(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1923,7 +1876,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... - @typing_extensions.final class Get(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1940,7 +1892,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... - @typing_extensions.final class GetWithDefault(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1957,7 +1908,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... - @typing_extensions.final class GetOption(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1974,7 +1924,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... - @typing_extensions.final class GetAll(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1996,7 +1945,6 @@ class ConfigRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_prefix", b"_prefix"] ) -> typing_extensions.Literal["prefix"] | None: ... - @typing_extensions.final class Unset(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2013,7 +1961,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... - @typing_extensions.final class IsModifiable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2119,7 +2066,6 @@ class ConfigRequest(google.protobuf.message.Message): global___ConfigRequest = ConfigRequest -@typing_extensions.final class ConfigResponse(google.protobuf.message.Message): """Response to the config request. Next ID: 5 @@ -2177,13 +2123,11 @@ class ConfigResponse(google.protobuf.message.Message): global___ConfigResponse = ConfigResponse -@typing_extensions.final class AddArtifactsRequest(google.protobuf.message.Message): """Request to transfer client-local artifacts.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ArtifactChunk(google.protobuf.message.Message): """A chunk of an Artifact.""" @@ -2205,7 +2149,6 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["crc", b"crc", "data", b"data"] ) -> None: ... - @typing_extensions.final class SingleChunkArtifact(google.protobuf.message.Message): """An artifact that is contained in a single `ArtifactChunk`. Generally, this message represents tiny artifacts such as REPL-generated class files. @@ -2239,7 +2182,6 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["data", b"data", "name", b"name"] ) -> None: ... - @typing_extensions.final class Batch(google.protobuf.message.Message): """A number of `SingleChunkArtifact` batched into a single RPC.""" @@ -2262,7 +2204,6 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["artifacts", b"artifacts"] ) -> None: ... - @typing_extensions.final class BeginChunkedArtifact(google.protobuf.message.Message): """Signals the beginning/start of a chunked artifact. A large artifact is transferred through a payload of `BeginChunkedArtifact` followed by a @@ -2432,7 +2373,6 @@ class AddArtifactsRequest(google.protobuf.message.Message): global___AddArtifactsRequest = AddArtifactsRequest -@typing_extensions.final class AddArtifactsResponse(google.protobuf.message.Message): """Response to adding an artifact. Contains relevant metadata to verify successful transfer of artifact(s). @@ -2441,7 +2381,6 @@ class AddArtifactsResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ArtifactSummary(google.protobuf.message.Message): """Metadata of an artifact.""" @@ -2506,7 +2445,6 @@ class AddArtifactsResponse(google.protobuf.message.Message): global___AddArtifactsResponse = AddArtifactsResponse -@typing_extensions.final class ArtifactStatusesRequest(google.protobuf.message.Message): """Request to get current statuses of artifacts at the server side.""" @@ -2607,7 +2545,6 @@ class ArtifactStatusesRequest(google.protobuf.message.Message): global___ArtifactStatusesRequest = ArtifactStatusesRequest -@typing_extensions.final class ArtifactStatusesResponse(google.protobuf.message.Message): """Response to checking artifact statuses. Next ID: 4 @@ -2615,7 +2552,6 @@ class ArtifactStatusesResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class StatusesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2637,7 +2573,6 @@ class ArtifactStatusesResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class ArtifactStatus(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2693,7 +2628,6 @@ class ArtifactStatusesResponse(google.protobuf.message.Message): global___ArtifactStatusesResponse = ArtifactStatusesResponse -@typing_extensions.final class InterruptRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2706,7 +2640,7 @@ class InterruptRequest(google.protobuf.message.Message): InterruptRequest._InterruptType.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor INTERRUPT_TYPE_UNSPECIFIED: InterruptRequest._InterruptType.ValueType # 0 INTERRUPT_TYPE_ALL: InterruptRequest._InterruptType.ValueType # 1 @@ -2835,7 +2769,6 @@ class InterruptRequest(google.protobuf.message.Message): global___InterruptRequest = InterruptRequest -@typing_extensions.final class InterruptResponse(google.protobuf.message.Message): """Next ID: 4""" @@ -2876,7 +2809,6 @@ class InterruptResponse(google.protobuf.message.Message): global___InterruptResponse = InterruptResponse -@typing_extensions.final class ReattachOptions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2902,7 +2834,6 @@ class ReattachOptions(google.protobuf.message.Message): global___ReattachOptions = ReattachOptions -@typing_extensions.final class ReattachExecuteRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3021,11 +2952,9 @@ class ReattachExecuteRequest(google.protobuf.message.Message): global___ReattachExecuteRequest = ReattachExecuteRequest -@typing_extensions.final class ReleaseExecuteRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ReleaseAll(google.protobuf.message.Message): """Release and close operation completely. This will also interrupt the query if it is running execution, and wait for it to be torn down. @@ -3037,7 +2966,6 @@ class ReleaseExecuteRequest(google.protobuf.message.Message): self, ) -> None: ... - @typing_extensions.final class ReleaseUntil(google.protobuf.message.Message): """Release all responses from the operation response stream up to and including the response with the given by response_id. @@ -3174,7 +3102,6 @@ class ReleaseExecuteRequest(google.protobuf.message.Message): global___ReleaseExecuteRequest = ReleaseExecuteRequest -@typing_extensions.final class ReleaseExecuteResponse(google.protobuf.message.Message): """Next ID: 4""" @@ -3226,7 +3153,6 @@ class ReleaseExecuteResponse(google.protobuf.message.Message): global___ReleaseExecuteResponse = ReleaseExecuteResponse -@typing_extensions.final class ReleaseSessionRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3288,7 +3214,6 @@ class ReleaseSessionRequest(google.protobuf.message.Message): global___ReleaseSessionRequest = ReleaseSessionRequest -@typing_extensions.final class ReleaseSessionResponse(google.protobuf.message.Message): """Next ID: 3""" @@ -3317,7 +3242,6 @@ class ReleaseSessionResponse(google.protobuf.message.Message): global___ReleaseSessionResponse = ReleaseSessionResponse -@typing_extensions.final class FetchErrorDetailsRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3406,13 +3330,11 @@ class FetchErrorDetailsRequest(google.protobuf.message.Message): global___FetchErrorDetailsRequest = FetchErrorDetailsRequest -@typing_extensions.final class FetchErrorDetailsResponse(google.protobuf.message.Message): """Next ID: 5""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class StackTraceElement(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3461,7 +3383,6 @@ class FetchErrorDetailsResponse(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_file_name", b"_file_name"] ) -> typing_extensions.Literal["file_name"] | None: ... - @typing_extensions.final class QueryContext(google.protobuf.message.Message): """QueryContext defines the schema for the query context of a SparkThrowable. It helps users understand where the error occurs while executing queries. @@ -3478,7 +3399,7 @@ class FetchErrorDetailsResponse(google.protobuf.message.Message): FetchErrorDetailsResponse.QueryContext._ContextType.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SQL: FetchErrorDetailsResponse.QueryContext._ContextType.ValueType # 0 DATAFRAME: FetchErrorDetailsResponse.QueryContext._ContextType.ValueType # 1 @@ -3552,13 +3473,11 @@ class FetchErrorDetailsResponse(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class SparkThrowable(google.protobuf.message.Message): """SparkThrowable defines the schema for SparkThrowable exceptions.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class MessageParametersEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3648,7 +3567,6 @@ class FetchErrorDetailsResponse(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_sql_state", b"_sql_state"] ) -> typing_extensions.Literal["sql_state"] | None: ... - @typing_extensions.final class Error(google.protobuf.message.Message): """Error defines the schema for the representing exception.""" diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.pyi b/python/pyspark/sql/connect/proto/catalog_pb2.pyi index 9ead7e04daf0..2879b44618b5 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.pyi +++ b/python/pyspark/sql/connect/proto/catalog_pb2.pyi @@ -50,7 +50,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Catalog(google.protobuf.message.Message): """Catalog messages are marked as unstable.""" @@ -318,7 +317,6 @@ class Catalog(google.protobuf.message.Message): global___Catalog = Catalog -@typing_extensions.final class CurrentDatabase(google.protobuf.message.Message): """See `spark.catalog.currentDatabase`""" @@ -330,7 +328,6 @@ class CurrentDatabase(google.protobuf.message.Message): global___CurrentDatabase = CurrentDatabase -@typing_extensions.final class SetCurrentDatabase(google.protobuf.message.Message): """See `spark.catalog.setCurrentDatabase`""" @@ -348,7 +345,6 @@ class SetCurrentDatabase(google.protobuf.message.Message): global___SetCurrentDatabase = SetCurrentDatabase -@typing_extensions.final class ListDatabases(google.protobuf.message.Message): """See `spark.catalog.listDatabases`""" @@ -374,7 +370,6 @@ class ListDatabases(google.protobuf.message.Message): global___ListDatabases = ListDatabases -@typing_extensions.final class ListTables(google.protobuf.message.Message): """See `spark.catalog.listTables`""" @@ -429,7 +424,6 @@ class ListTables(google.protobuf.message.Message): global___ListTables = ListTables -@typing_extensions.final class ListFunctions(google.protobuf.message.Message): """See `spark.catalog.listFunctions`""" @@ -484,7 +478,6 @@ class ListFunctions(google.protobuf.message.Message): global___ListFunctions = ListFunctions -@typing_extensions.final class ListColumns(google.protobuf.message.Message): """See `spark.catalog.listColumns`""" @@ -517,7 +510,6 @@ class ListColumns(google.protobuf.message.Message): global___ListColumns = ListColumns -@typing_extensions.final class GetDatabase(google.protobuf.message.Message): """See `spark.catalog.getDatabase`""" @@ -535,7 +527,6 @@ class GetDatabase(google.protobuf.message.Message): global___GetDatabase = GetDatabase -@typing_extensions.final class GetTable(google.protobuf.message.Message): """See `spark.catalog.getTable`""" @@ -568,7 +559,6 @@ class GetTable(google.protobuf.message.Message): global___GetTable = GetTable -@typing_extensions.final class GetFunction(google.protobuf.message.Message): """See `spark.catalog.getFunction`""" @@ -601,7 +591,6 @@ class GetFunction(google.protobuf.message.Message): global___GetFunction = GetFunction -@typing_extensions.final class DatabaseExists(google.protobuf.message.Message): """See `spark.catalog.databaseExists`""" @@ -619,7 +608,6 @@ class DatabaseExists(google.protobuf.message.Message): global___DatabaseExists = DatabaseExists -@typing_extensions.final class TableExists(google.protobuf.message.Message): """See `spark.catalog.tableExists`""" @@ -652,7 +640,6 @@ class TableExists(google.protobuf.message.Message): global___TableExists = TableExists -@typing_extensions.final class FunctionExists(google.protobuf.message.Message): """See `spark.catalog.functionExists`""" @@ -685,13 +672,11 @@ class FunctionExists(google.protobuf.message.Message): global___FunctionExists = FunctionExists -@typing_extensions.final class CreateExternalTable(google.protobuf.message.Message): """See `spark.catalog.createExternalTable`""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -790,13 +775,11 @@ class CreateExternalTable(google.protobuf.message.Message): global___CreateExternalTable = CreateExternalTable -@typing_extensions.final class CreateTable(google.protobuf.message.Message): """See `spark.catalog.createTable`""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -911,7 +894,6 @@ class CreateTable(google.protobuf.message.Message): global___CreateTable = CreateTable -@typing_extensions.final class DropTempView(google.protobuf.message.Message): """See `spark.catalog.dropTempView`""" @@ -931,7 +913,6 @@ class DropTempView(google.protobuf.message.Message): global___DropTempView = DropTempView -@typing_extensions.final class DropGlobalTempView(google.protobuf.message.Message): """See `spark.catalog.dropGlobalTempView`""" @@ -951,7 +932,6 @@ class DropGlobalTempView(google.protobuf.message.Message): global___DropGlobalTempView = DropGlobalTempView -@typing_extensions.final class RecoverPartitions(google.protobuf.message.Message): """See `spark.catalog.recoverPartitions`""" @@ -971,7 +951,6 @@ class RecoverPartitions(google.protobuf.message.Message): global___RecoverPartitions = RecoverPartitions -@typing_extensions.final class IsCached(google.protobuf.message.Message): """See `spark.catalog.isCached`""" @@ -991,7 +970,6 @@ class IsCached(google.protobuf.message.Message): global___IsCached = IsCached -@typing_extensions.final class CacheTable(google.protobuf.message.Message): """See `spark.catalog.cacheTable`""" @@ -1033,7 +1011,6 @@ class CacheTable(google.protobuf.message.Message): global___CacheTable = CacheTable -@typing_extensions.final class UncacheTable(google.protobuf.message.Message): """See `spark.catalog.uncacheTable`""" @@ -1053,7 +1030,6 @@ class UncacheTable(google.protobuf.message.Message): global___UncacheTable = UncacheTable -@typing_extensions.final class ClearCache(google.protobuf.message.Message): """See `spark.catalog.clearCache`""" @@ -1065,7 +1041,6 @@ class ClearCache(google.protobuf.message.Message): global___ClearCache = ClearCache -@typing_extensions.final class RefreshTable(google.protobuf.message.Message): """See `spark.catalog.refreshTable`""" @@ -1085,7 +1060,6 @@ class RefreshTable(google.protobuf.message.Message): global___RefreshTable = RefreshTable -@typing_extensions.final class RefreshByPath(google.protobuf.message.Message): """See `spark.catalog.refreshByPath`""" @@ -1103,7 +1077,6 @@ class RefreshByPath(google.protobuf.message.Message): global___RefreshByPath = RefreshByPath -@typing_extensions.final class CurrentCatalog(google.protobuf.message.Message): """See `spark.catalog.currentCatalog`""" @@ -1115,7 +1088,6 @@ class CurrentCatalog(google.protobuf.message.Message): global___CurrentCatalog = CurrentCatalog -@typing_extensions.final class SetCurrentCatalog(google.protobuf.message.Message): """See `spark.catalog.setCurrentCatalog`""" @@ -1135,7 +1107,6 @@ class SetCurrentCatalog(google.protobuf.message.Message): global___SetCurrentCatalog = SetCurrentCatalog -@typing_extensions.final class ListCatalogs(google.protobuf.message.Message): """See `spark.catalog.listCatalogs`""" diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index bb5690556101..b57a2a6c4d68 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -60,7 +60,7 @@ class _StreamingQueryEventType: class _StreamingQueryEventTypeEnumTypeWrapper( google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[_StreamingQueryEventType.ValueType], builtins.type, -): +): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor QUERY_PROGRESS_UNSPECIFIED: _StreamingQueryEventType.ValueType # 0 QUERY_PROGRESS_EVENT: _StreamingQueryEventType.ValueType # 1 @@ -81,7 +81,6 @@ QUERY_TERMINATED_EVENT: StreamingQueryEventType.ValueType # 2 QUERY_IDLE_EVENT: StreamingQueryEventType.ValueType # 3 global___StreamingQueryEventType = StreamingQueryEventType -@typing_extensions.final class Command(google.protobuf.message.Message): """A [[Command]] is an operation that is executed by the server that does not directly consume or produce a relational result. @@ -256,7 +255,6 @@ class Command(google.protobuf.message.Message): global___Command = Command -@typing_extensions.final class SqlCommand(google.protobuf.message.Message): """A SQL Command is used to trigger the eager evaluation of SQL commands in Spark. @@ -268,7 +266,6 @@ class SqlCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ArgsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -290,7 +287,6 @@ class SqlCommand(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class NamedArgumentsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -390,7 +386,6 @@ class SqlCommand(google.protobuf.message.Message): global___SqlCommand = SqlCommand -@typing_extensions.final class CreateDataFrameViewCommand(google.protobuf.message.Message): """A command that can create DataFrame global temp view or local temp view.""" @@ -433,7 +428,6 @@ class CreateDataFrameViewCommand(google.protobuf.message.Message): global___CreateDataFrameViewCommand = CreateDataFrameViewCommand -@typing_extensions.final class WriteOperation(google.protobuf.message.Message): """As writes are not directly handled during analysis and planning, they are modeled as commands.""" @@ -448,7 +442,7 @@ class WriteOperation(google.protobuf.message.Message): WriteOperation._SaveMode.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SAVE_MODE_UNSPECIFIED: WriteOperation._SaveMode.ValueType # 0 SAVE_MODE_APPEND: WriteOperation._SaveMode.ValueType # 1 @@ -463,7 +457,6 @@ class WriteOperation(google.protobuf.message.Message): SAVE_MODE_ERROR_IF_EXISTS: WriteOperation.SaveMode.ValueType # 3 SAVE_MODE_IGNORE: WriteOperation.SaveMode.ValueType # 4 - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -481,7 +474,6 @@ class WriteOperation(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class SaveTable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -494,7 +486,7 @@ class WriteOperation(google.protobuf.message.Message): WriteOperation.SaveTable._TableSaveMethod.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor TABLE_SAVE_METHOD_UNSPECIFIED: WriteOperation.SaveTable._TableSaveMethod.ValueType # 0 TABLE_SAVE_METHOD_SAVE_AS_TABLE: WriteOperation.SaveTable._TableSaveMethod.ValueType # 1 @@ -524,7 +516,6 @@ class WriteOperation(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class BucketBy(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -664,7 +655,6 @@ class WriteOperation(google.protobuf.message.Message): global___WriteOperation = WriteOperation -@typing_extensions.final class WriteOperationV2(google.protobuf.message.Message): """As writes are not directly handled during analysis and planning, they are modeled as commands.""" @@ -679,7 +669,7 @@ class WriteOperationV2(google.protobuf.message.Message): WriteOperationV2._Mode.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor MODE_UNSPECIFIED: WriteOperationV2._Mode.ValueType # 0 MODE_CREATE: WriteOperationV2._Mode.ValueType # 1 @@ -698,7 +688,6 @@ class WriteOperationV2(google.protobuf.message.Message): MODE_REPLACE: WriteOperationV2.Mode.ValueType # 5 MODE_CREATE_OR_REPLACE: WriteOperationV2.Mode.ValueType # 6 - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -716,7 +705,6 @@ class WriteOperationV2(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class TablePropertiesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -839,7 +827,6 @@ class WriteOperationV2(google.protobuf.message.Message): global___WriteOperationV2 = WriteOperationV2 -@typing_extensions.final class WriteStreamOperationStart(google.protobuf.message.Message): """Starts write stream operation as streaming query. Query ID and Run ID of the streaming query are returned. @@ -847,7 +834,6 @@ class WriteStreamOperationStart(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1004,7 +990,6 @@ class WriteStreamOperationStart(google.protobuf.message.Message): global___WriteStreamOperationStart = WriteStreamOperationStart -@typing_extensions.final class StreamingForeachFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1048,7 +1033,6 @@ class StreamingForeachFunction(google.protobuf.message.Message): global___StreamingForeachFunction = StreamingForeachFunction -@typing_extensions.final class WriteStreamOperationStartResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1102,7 +1086,6 @@ class WriteStreamOperationStartResult(google.protobuf.message.Message): global___WriteStreamOperationStartResult = WriteStreamOperationStartResult -@typing_extensions.final class StreamingQueryInstanceId(google.protobuf.message.Message): """A tuple that uniquely identifies an instance of streaming query run. It consists of `id` that persists across the streaming runs and `run_id` that changes between each run of the @@ -1135,13 +1118,11 @@ class StreamingQueryInstanceId(google.protobuf.message.Message): global___StreamingQueryInstanceId = StreamingQueryInstanceId -@typing_extensions.final class StreamingQueryCommand(google.protobuf.message.Message): """Commands for a streaming query.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ExplainCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1159,7 +1140,6 @@ class StreamingQueryCommand(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["extended", b"extended"] ) -> None: ... - @typing_extensions.final class AwaitTerminationCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1297,13 +1277,11 @@ class StreamingQueryCommand(google.protobuf.message.Message): global___StreamingQueryCommand = StreamingQueryCommand -@typing_extensions.final class StreamingQueryCommandResult(google.protobuf.message.Message): """Response for commands on a streaming query.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class StatusResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1338,7 +1316,6 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class RecentProgressResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1358,7 +1335,6 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): field_name: typing_extensions.Literal["recent_progress_json", b"recent_progress_json"], ) -> None: ... - @typing_extensions.final class ExplainResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1374,7 +1350,6 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... - @typing_extensions.final class ExceptionResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1444,7 +1419,6 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_stack_trace", b"_stack_trace"] ) -> typing_extensions.Literal["stack_trace"] | None: ... - @typing_extensions.final class AwaitTerminationResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1537,13 +1511,11 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): global___StreamingQueryCommandResult = StreamingQueryCommandResult -@typing_extensions.final class StreamingQueryManagerCommand(google.protobuf.message.Message): """Commands for the streaming query manager.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class AwaitAnyTerminationCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1571,7 +1543,6 @@ class StreamingQueryManagerCommand(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_timeout_ms", b"_timeout_ms"] ) -> typing_extensions.Literal["timeout_ms"] | None: ... - @typing_extensions.final class StreamingQueryListenerCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1722,13 +1693,11 @@ class StreamingQueryManagerCommand(google.protobuf.message.Message): global___StreamingQueryManagerCommand = StreamingQueryManagerCommand -@typing_extensions.final class StreamingQueryManagerCommandResult(google.protobuf.message.Message): """Response for commands on the streaming query manager.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ActiveResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1751,7 +1720,6 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["active_queries", b"active_queries"] ) -> None: ... - @typing_extensions.final class StreamingQueryInstance(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1780,7 +1748,6 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_name", b"_name"] ) -> typing_extensions.Literal["name"] | None: ... - @typing_extensions.final class AwaitAnyTerminationResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1795,7 +1762,6 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["terminated", b"terminated"] ) -> None: ... - @typing_extensions.final class StreamingQueryListenerInstance(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1810,7 +1776,6 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["listener_payload", b"listener_payload"] ) -> None: ... - @typing_extensions.final class ListStreamingQueryListenerResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1923,7 +1888,6 @@ class StreamingQueryManagerCommandResult(google.protobuf.message.Message): global___StreamingQueryManagerCommandResult = StreamingQueryManagerCommandResult -@typing_extensions.final class StreamingQueryListenerBusCommand(google.protobuf.message.Message): """The protocol for client-side StreamingQueryListener. This command will only be set when either the first listener is added to the client, or the last @@ -1975,7 +1939,6 @@ class StreamingQueryListenerBusCommand(google.protobuf.message.Message): global___StreamingQueryListenerBusCommand = StreamingQueryListenerBusCommand -@typing_extensions.final class StreamingQueryListenerEvent(google.protobuf.message.Message): """The protocol for the returned events in the long-running response channel.""" @@ -2002,7 +1965,6 @@ class StreamingQueryListenerEvent(google.protobuf.message.Message): global___StreamingQueryListenerEvent = StreamingQueryListenerEvent -@typing_extensions.final class StreamingQueryListenerEventsResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2050,7 +2012,6 @@ class StreamingQueryListenerEventsResult(google.protobuf.message.Message): global___StreamingQueryListenerEventsResult = StreamingQueryListenerEventsResult -@typing_extensions.final class GetResourcesCommand(google.protobuf.message.Message): """Command to get the output of 'SparkContext.resources'""" @@ -2062,13 +2023,11 @@ class GetResourcesCommand(google.protobuf.message.Message): global___GetResourcesCommand = GetResourcesCommand -@typing_extensions.final class GetResourcesCommandResult(google.protobuf.message.Message): """Response for command 'GetResourcesCommand'.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ResourcesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2111,7 +2070,6 @@ class GetResourcesCommandResult(google.protobuf.message.Message): global___GetResourcesCommandResult = GetResourcesCommandResult -@typing_extensions.final class CreateResourceProfileCommand(google.protobuf.message.Message): """Command to create ResourceProfile""" @@ -2133,7 +2091,6 @@ class CreateResourceProfileCommand(google.protobuf.message.Message): global___CreateResourceProfileCommand = CreateResourceProfileCommand -@typing_extensions.final class CreateResourceProfileCommandResult(google.protobuf.message.Message): """Response for command 'CreateResourceProfileCommand'.""" diff --git a/python/pyspark/sql/connect/proto/common_pb2.pyi b/python/pyspark/sql/connect/proto/common_pb2.pyi index 9845ede2daa2..163781b41998 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.pyi +++ b/python/pyspark/sql/connect/proto/common_pb2.pyi @@ -48,7 +48,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class StorageLevel(google.protobuf.message.Message): """StorageLevel for persisting Datasets/Tables.""" @@ -96,7 +95,6 @@ class StorageLevel(google.protobuf.message.Message): global___StorageLevel = StorageLevel -@typing_extensions.final class ResourceInformation(google.protobuf.message.Message): """ResourceInformation to hold information about a type of Resource. The corresponding class is 'org.apache.spark.resource.ResourceInformation' @@ -125,7 +123,6 @@ class ResourceInformation(google.protobuf.message.Message): global___ResourceInformation = ResourceInformation -@typing_extensions.final class ExecutorResourceRequest(google.protobuf.message.Message): """An executor resource request.""" @@ -192,7 +189,6 @@ class ExecutorResourceRequest(google.protobuf.message.Message): global___ExecutorResourceRequest = ExecutorResourceRequest -@typing_extensions.final class TaskResourceRequest(google.protobuf.message.Message): """A task resource request.""" @@ -221,11 +217,9 @@ class TaskResourceRequest(google.protobuf.message.Message): global___TaskResourceRequest = TaskResourceRequest -@typing_extensions.final class ResourceProfile(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ExecutorResourcesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -247,7 +241,6 @@ class ResourceProfile(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class TaskResourcesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi b/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi index 4a068a7980b3..1be966ff1e40 100644 --- a/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi +++ b/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi @@ -47,7 +47,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class ExamplePluginRelation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -72,7 +71,6 @@ class ExamplePluginRelation(google.protobuf.message.Message): global___ExamplePluginRelation = ExamplePluginRelation -@typing_extensions.final class ExamplePluginExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -97,7 +95,6 @@ class ExamplePluginExpression(google.protobuf.message.Message): global___ExamplePluginExpression = ExamplePluginExpression -@typing_extensions.final class ExamplePluginCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index 20f7f6115063..183a839da920 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -51,7 +51,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Expression(google.protobuf.message.Message): """Expression used to refer to fields, functions and similar. This can be used everywhere expressions in SQL appear. @@ -59,13 +58,11 @@ class Expression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Window(google.protobuf.message.Message): """Expression for the OVER clause or WINDOW clause.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class WindowFrame(google.protobuf.message.Message): """The window frame""" @@ -80,7 +77,7 @@ class Expression(google.protobuf.message.Message): Expression.Window.WindowFrame._FrameType.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor FRAME_TYPE_UNDEFINED: Expression.Window.WindowFrame._FrameType.ValueType # 0 FRAME_TYPE_ROW: Expression.Window.WindowFrame._FrameType.ValueType # 1 @@ -99,7 +96,6 @@ class Expression(google.protobuf.message.Message): All rows having the same 'ORDER BY' ordering are considered as peers. """ - @typing_extensions.final class FrameBoundary(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -236,7 +232,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class SortOrder(google.protobuf.message.Message): """SortOrder is used to specify the data ordering, it is normally used in Sort and Window. It is an unevaluable expression and cannot be evaluated, so can not be used in Projection. @@ -253,7 +248,7 @@ class Expression(google.protobuf.message.Message): Expression.SortOrder._SortDirection.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SORT_DIRECTION_UNSPECIFIED: Expression.SortOrder._SortDirection.ValueType # 0 SORT_DIRECTION_ASCENDING: Expression.SortOrder._SortDirection.ValueType # 1 @@ -273,7 +268,7 @@ class Expression(google.protobuf.message.Message): Expression.SortOrder._NullOrdering.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SORT_NULLS_UNSPECIFIED: Expression.SortOrder._NullOrdering.ValueType # 0 SORT_NULLS_FIRST: Expression.SortOrder._NullOrdering.ValueType # 1 @@ -311,7 +306,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Cast(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -324,7 +318,7 @@ class Expression(google.protobuf.message.Message): Expression.Cast._EvalMode.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor EVAL_MODE_UNSPECIFIED: Expression.Cast._EvalMode.ValueType # 0 EVAL_MODE_LEGACY: Expression.Cast._EvalMode.ValueType # 1 @@ -390,11 +384,9 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["cast_to_type", b"cast_to_type"] ) -> typing_extensions.Literal["type", "type_str"] | None: ... - @typing_extensions.final class Literal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Decimal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -453,7 +445,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_scale", b"_scale"] ) -> typing_extensions.Literal["scale"] | None: ... - @typing_extensions.final class CalendarInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -477,7 +468,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Array(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -507,7 +497,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Map(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -559,7 +548,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Struct(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -784,7 +772,6 @@ class Expression(google.protobuf.message.Message): | None ): ... - @typing_extensions.final class UnresolvedAttribute(google.protobuf.message.Message): """An unresolved attribute that is not explicitly bound to a specific column, but the column is resolved during analysis by name. @@ -848,7 +835,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_plan_id", b"_plan_id"] ) -> typing_extensions.Literal["plan_id"] | None: ... - @typing_extensions.final class UnresolvedFunction(google.protobuf.message.Message): """An unresolved function is not explicitly bound to one explicit function, but the function is resolved during analysis following Sparks name resolution rules. @@ -899,7 +885,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class ExpressionString(google.protobuf.message.Message): """Expression as string.""" @@ -917,7 +902,6 @@ class Expression(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["expression", b"expression"] ) -> None: ... - @typing_extensions.final class UnresolvedStar(google.protobuf.message.Message): """UnresolvedStar is used to expand all the fields of a relation or struct.""" @@ -974,7 +958,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_unparsed_target", b"_unparsed_target"] ) -> typing_extensions.Literal["unparsed_target"] | None: ... - @typing_extensions.final class UnresolvedRegex(google.protobuf.message.Message): """Represents all of the input attributes to a given relational operator, for example in "SELECT `(id)?+.+` FROM ...". @@ -1008,7 +991,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_plan_id", b"_plan_id"] ) -> typing_extensions.Literal["plan_id"] | None: ... - @typing_extensions.final class UnresolvedExtractValue(google.protobuf.message.Message): """Extracts a value or values from an Expression""" @@ -1041,7 +1023,6 @@ class Expression(google.protobuf.message.Message): field_name: typing_extensions.Literal["child", b"child", "extraction", b"extraction"], ) -> None: ... - @typing_extensions.final class UpdateFields(google.protobuf.message.Message): """Add, replace or drop a field of `StructType` expression by name.""" @@ -1086,7 +1067,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Alias(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1129,7 +1109,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_metadata", b"_metadata"] ) -> typing_extensions.Literal["metadata"] | None: ... - @typing_extensions.final class LambdaFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1166,7 +1145,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class UnresolvedNamedLambdaVariable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1381,7 +1359,6 @@ class Expression(google.protobuf.message.Message): global___Expression = Expression -@typing_extensions.final class CommonInlineUserDefinedFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1454,7 +1431,6 @@ class CommonInlineUserDefinedFunction(google.protobuf.message.Message): global___CommonInlineUserDefinedFunction = CommonInlineUserDefinedFunction -@typing_extensions.final class PythonUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1498,7 +1474,6 @@ class PythonUDF(google.protobuf.message.Message): global___PythonUDF = PythonUDF -@typing_extensions.final class ScalarScalaUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1548,7 +1523,6 @@ class ScalarScalaUDF(google.protobuf.message.Message): global___ScalarScalaUDF = ScalarScalaUDF -@typing_extensions.final class JavaUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1594,7 +1568,6 @@ class JavaUDF(google.protobuf.message.Message): global___JavaUDF = JavaUDF -@typing_extensions.final class CallFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1622,7 +1595,6 @@ class CallFunction(google.protobuf.message.Message): global___CallFunction = CallFunction -@typing_extensions.final class NamedArgumentExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index 9df01405453d..db9609eebb85 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -53,7 +53,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Relation(google.protobuf.message.Message): """The main [[Relation]] type. Fundamentally, a relation is a typed container that has exactly one explicit relation type set. @@ -588,7 +587,6 @@ class Relation(google.protobuf.message.Message): global___Relation = Relation -@typing_extensions.final class Unknown(google.protobuf.message.Message): """Used for testing purposes only.""" @@ -600,7 +598,6 @@ class Unknown(google.protobuf.message.Message): global___Unknown = Unknown -@typing_extensions.final class RelationCommon(google.protobuf.message.Message): """Common metadata of all relations.""" @@ -633,13 +630,11 @@ class RelationCommon(google.protobuf.message.Message): global___RelationCommon = RelationCommon -@typing_extensions.final class SQL(google.protobuf.message.Message): """Relation that uses a SQL query to generate the output.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ArgsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -661,7 +656,6 @@ class SQL(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class NamedArgumentsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -761,7 +755,6 @@ class SQL(google.protobuf.message.Message): global___SQL = SQL -@typing_extensions.final class Read(google.protobuf.message.Message): """Relation that reads from a file / table or other data source. Does not have additional inputs. @@ -769,11 +762,9 @@ class Read(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class NamedTable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -813,11 +804,9 @@ class Read(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class DataSource(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -962,7 +951,6 @@ class Read(google.protobuf.message.Message): global___Read = Read -@typing_extensions.final class Project(google.protobuf.message.Message): """Projection of a bag of expressions for a given input relation. @@ -1004,7 +992,6 @@ class Project(google.protobuf.message.Message): global___Project = Project -@typing_extensions.final class Filter(google.protobuf.message.Message): """Relation that applies a boolean expression `condition` on each row of `input` to produce the output result. @@ -1035,7 +1022,6 @@ class Filter(google.protobuf.message.Message): global___Filter = Filter -@typing_extensions.final class Join(google.protobuf.message.Message): """Relation of type [[Join]]. @@ -1051,7 +1037,7 @@ class Join(google.protobuf.message.Message): class _JoinTypeEnumTypeWrapper( google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[Join._JoinType.ValueType], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor JOIN_TYPE_UNSPECIFIED: Join._JoinType.ValueType # 0 JOIN_TYPE_INNER: Join._JoinType.ValueType # 1 @@ -1072,7 +1058,6 @@ class Join(google.protobuf.message.Message): JOIN_TYPE_LEFT_SEMI: Join.JoinType.ValueType # 6 JOIN_TYPE_CROSS: Join.JoinType.ValueType # 7 - @typing_extensions.final class JoinDataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1178,7 +1163,6 @@ class Join(google.protobuf.message.Message): global___Join = Join -@typing_extensions.final class SetOperation(google.protobuf.message.Message): """Relation of type [[SetOperation]]""" @@ -1193,7 +1177,7 @@ class SetOperation(google.protobuf.message.Message): SetOperation._SetOpType.ValueType ], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor SET_OP_TYPE_UNSPECIFIED: SetOperation._SetOpType.ValueType # 0 SET_OP_TYPE_INTERSECT: SetOperation._SetOpType.ValueType # 1 @@ -1306,7 +1290,6 @@ class SetOperation(google.protobuf.message.Message): global___SetOperation = SetOperation -@typing_extensions.final class Limit(google.protobuf.message.Message): """Relation of type [[Limit]] that is used to `limit` rows from the input relation.""" @@ -1334,7 +1317,6 @@ class Limit(google.protobuf.message.Message): global___Limit = Limit -@typing_extensions.final class Offset(google.protobuf.message.Message): """Relation of type [[Offset]] that is used to read rows staring from the `offset` on the input relation. @@ -1364,7 +1346,6 @@ class Offset(google.protobuf.message.Message): global___Offset = Offset -@typing_extensions.final class Tail(google.protobuf.message.Message): """Relation of type [[Tail]] that is used to fetch `limit` rows from the last of the input relation.""" @@ -1392,7 +1373,6 @@ class Tail(google.protobuf.message.Message): global___Tail = Tail -@typing_extensions.final class Aggregate(google.protobuf.message.Message): """Relation of type [[Aggregate]].""" @@ -1405,7 +1385,7 @@ class Aggregate(google.protobuf.message.Message): class _GroupTypeEnumTypeWrapper( google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[Aggregate._GroupType.ValueType], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor GROUP_TYPE_UNSPECIFIED: Aggregate._GroupType.ValueType # 0 GROUP_TYPE_GROUPBY: Aggregate._GroupType.ValueType # 1 @@ -1422,7 +1402,6 @@ class Aggregate(google.protobuf.message.Message): GROUP_TYPE_PIVOT: Aggregate.GroupType.ValueType # 4 GROUP_TYPE_GROUPING_SETS: Aggregate.GroupType.ValueType # 5 - @typing_extensions.final class Pivot(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1458,7 +1437,6 @@ class Aggregate(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["col", b"col", "values", b"values"] ) -> None: ... - @typing_extensions.final class GroupingSets(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1556,7 +1534,6 @@ class Aggregate(google.protobuf.message.Message): global___Aggregate = Aggregate -@typing_extensions.final class Sort(google.protobuf.message.Message): """Relation of type [[Sort]].""" @@ -1612,7 +1589,6 @@ class Sort(google.protobuf.message.Message): global___Sort = Sort -@typing_extensions.final class Drop(google.protobuf.message.Message): """Drop specified columns.""" @@ -1656,7 +1632,6 @@ class Drop(google.protobuf.message.Message): global___Drop = Drop -@typing_extensions.final class Deduplicate(google.protobuf.message.Message): """Relation of type [[Deduplicate]] which have duplicate rows removed, could consider either only the subset of columns or all the columns. @@ -1738,7 +1713,6 @@ class Deduplicate(google.protobuf.message.Message): global___Deduplicate = Deduplicate -@typing_extensions.final class LocalRelation(google.protobuf.message.Message): """A relation that does not need to be qualified by name.""" @@ -1786,7 +1760,6 @@ class LocalRelation(google.protobuf.message.Message): global___LocalRelation = LocalRelation -@typing_extensions.final class CachedLocalRelation(google.protobuf.message.Message): """A local relation that has been cached already.""" @@ -1804,7 +1777,6 @@ class CachedLocalRelation(google.protobuf.message.Message): global___CachedLocalRelation = CachedLocalRelation -@typing_extensions.final class CachedRemoteRelation(google.protobuf.message.Message): """Represents a remote relation that has been cached on server.""" @@ -1824,7 +1796,6 @@ class CachedRemoteRelation(google.protobuf.message.Message): global___CachedRemoteRelation = CachedRemoteRelation -@typing_extensions.final class Sample(google.protobuf.message.Message): """Relation of type [[Sample]] that samples a fraction of the dataset.""" @@ -1909,7 +1880,6 @@ class Sample(google.protobuf.message.Message): global___Sample = Sample -@typing_extensions.final class Range(google.protobuf.message.Message): """Relation of type [[Range]] that generates a sequence of integers.""" @@ -1978,7 +1948,6 @@ class Range(google.protobuf.message.Message): global___Range = Range -@typing_extensions.final class SubqueryAlias(google.protobuf.message.Message): """Relation alias.""" @@ -2016,7 +1985,6 @@ class SubqueryAlias(google.protobuf.message.Message): global___SubqueryAlias = SubqueryAlias -@typing_extensions.final class Repartition(google.protobuf.message.Message): """Relation repartition.""" @@ -2064,7 +2032,6 @@ class Repartition(google.protobuf.message.Message): global___Repartition = Repartition -@typing_extensions.final class ShowString(google.protobuf.message.Message): """Compose the string representing rows for output. It will invoke 'Dataset.showString' to compute the results. @@ -2114,7 +2081,6 @@ class ShowString(google.protobuf.message.Message): global___ShowString = ShowString -@typing_extensions.final class HtmlString(google.protobuf.message.Message): """Compose the string representing rows for output. It will invoke 'Dataset.htmlString' to compute the results. @@ -2153,7 +2119,6 @@ class HtmlString(google.protobuf.message.Message): global___HtmlString = HtmlString -@typing_extensions.final class StatSummary(google.protobuf.message.Message): """Computes specified statistics for numeric and string columns. It will invoke 'Dataset.summary' (same as 'StatFunctions.summary') @@ -2201,7 +2166,6 @@ class StatSummary(google.protobuf.message.Message): global___StatSummary = StatSummary -@typing_extensions.final class StatDescribe(google.protobuf.message.Message): """Computes basic statistics for numeric and string columns, including count, mean, stddev, min, and max. If no columns are given, this function computes statistics for all numerical or @@ -2235,7 +2199,6 @@ class StatDescribe(google.protobuf.message.Message): global___StatDescribe = StatDescribe -@typing_extensions.final class StatCrosstab(google.protobuf.message.Message): """Computes a pair-wise frequency table of the given columns. Also known as a contingency table. It will invoke 'Dataset.stat.crosstab' (same as 'StatFunctions.crossTabulate') @@ -2277,7 +2240,6 @@ class StatCrosstab(google.protobuf.message.Message): global___StatCrosstab = StatCrosstab -@typing_extensions.final class StatCov(google.protobuf.message.Message): """Calculate the sample covariance of two numerical columns of a DataFrame. It will invoke 'Dataset.stat.cov' (same as 'StatFunctions.calculateCov') to compute the results. @@ -2312,7 +2274,6 @@ class StatCov(google.protobuf.message.Message): global___StatCov = StatCov -@typing_extensions.final class StatCorr(google.protobuf.message.Message): """Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson Correlation Coefficient. It will invoke 'Dataset.stat.corr' (same as @@ -2372,7 +2333,6 @@ class StatCorr(google.protobuf.message.Message): global___StatCorr = StatCorr -@typing_extensions.final class StatApproxQuantile(google.protobuf.message.Message): """Calculates the approximate quantiles of numerical columns of a DataFrame. It will invoke 'Dataset.stat.approxQuantile' (same as 'StatFunctions.approxQuantile') @@ -2435,7 +2395,6 @@ class StatApproxQuantile(google.protobuf.message.Message): global___StatApproxQuantile = StatApproxQuantile -@typing_extensions.final class StatFreqItems(google.protobuf.message.Message): """Finding frequent items for columns, possibly with false positives. It will invoke 'Dataset.stat.freqItems' (same as 'StatFunctions.freqItems') @@ -2484,7 +2443,6 @@ class StatFreqItems(google.protobuf.message.Message): global___StatFreqItems = StatFreqItems -@typing_extensions.final class StatSampleBy(google.protobuf.message.Message): """Returns a stratified sample without replacement based on the fraction given on each stratum. @@ -2494,7 +2452,6 @@ class StatSampleBy(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Fraction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2576,7 +2533,6 @@ class StatSampleBy(google.protobuf.message.Message): global___StatSampleBy = StatSampleBy -@typing_extensions.final class NAFill(google.protobuf.message.Message): """Replaces null values. It will invoke 'Dataset.na.fill' (same as 'DataFrameNaFunctions.fill') to compute the results. @@ -2635,7 +2591,6 @@ class NAFill(google.protobuf.message.Message): global___NAFill = NAFill -@typing_extensions.final class NADrop(google.protobuf.message.Message): """Drop rows containing null values. It will invoke 'Dataset.na.drop' (same as 'DataFrameNaFunctions.drop') to compute the results. @@ -2704,7 +2659,6 @@ class NADrop(google.protobuf.message.Message): global___NADrop = NADrop -@typing_extensions.final class NAReplace(google.protobuf.message.Message): """Replaces old values with the corresponding values. It will invoke 'Dataset.na.replace' (same as 'DataFrameNaFunctions.replace') @@ -2713,7 +2667,6 @@ class NAReplace(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Replacement(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2790,7 +2743,6 @@ class NAReplace(google.protobuf.message.Message): global___NAReplace = NAReplace -@typing_extensions.final class ToDF(google.protobuf.message.Message): """Rename columns on the input relation by the same length of names.""" @@ -2826,13 +2778,11 @@ class ToDF(google.protobuf.message.Message): global___ToDF = ToDF -@typing_extensions.final class WithColumnsRenamed(google.protobuf.message.Message): """Rename columns on the input relation by a map with name to name mapping.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class RenameColumnsMapEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2850,7 +2800,6 @@ class WithColumnsRenamed(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class Rename(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2915,7 +2864,6 @@ class WithColumnsRenamed(google.protobuf.message.Message): global___WithColumnsRenamed = WithColumnsRenamed -@typing_extensions.final class WithColumns(google.protobuf.message.Message): """Adding columns or replacing the existing columns that have the same names.""" @@ -2960,7 +2908,6 @@ class WithColumns(google.protobuf.message.Message): global___WithColumns = WithColumns -@typing_extensions.final class WithWatermark(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2993,7 +2940,6 @@ class WithWatermark(google.protobuf.message.Message): global___WithWatermark = WithWatermark -@typing_extensions.final class Hint(google.protobuf.message.Message): """Specify a hint over a relation. Hint should have a name and optional parameters.""" @@ -3039,13 +2985,11 @@ class Hint(google.protobuf.message.Message): global___Hint = Hint -@typing_extensions.final class Unpivot(google.protobuf.message.Message): """Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Values(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3127,7 +3071,6 @@ class Unpivot(google.protobuf.message.Message): global___Unpivot = Unpivot -@typing_extensions.final class ToSchema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3157,7 +3100,6 @@ class ToSchema(google.protobuf.message.Message): global___ToSchema = ToSchema -@typing_extensions.final class RepartitionByExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3216,7 +3158,6 @@ class RepartitionByExpression(google.protobuf.message.Message): global___RepartitionByExpression = RepartitionByExpression -@typing_extensions.final class MapPartitions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3288,7 +3229,6 @@ class MapPartitions(google.protobuf.message.Message): global___MapPartitions = MapPartitions -@typing_extensions.final class GroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3432,7 +3372,6 @@ class GroupMap(google.protobuf.message.Message): global___GroupMap = GroupMap -@typing_extensions.final class CoGroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3532,7 +3471,6 @@ class CoGroupMap(google.protobuf.message.Message): global___CoGroupMap = CoGroupMap -@typing_extensions.final class ApplyInPandasWithState(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3604,7 +3542,6 @@ class ApplyInPandasWithState(google.protobuf.message.Message): global___ApplyInPandasWithState = ApplyInPandasWithState -@typing_extensions.final class CommonInlineUserDefinedTableFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3661,7 +3598,6 @@ class CommonInlineUserDefinedTableFunction(google.protobuf.message.Message): global___CommonInlineUserDefinedTableFunction = CommonInlineUserDefinedTableFunction -@typing_extensions.final class PythonUDTF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3713,7 +3649,6 @@ class PythonUDTF(google.protobuf.message.Message): global___PythonUDTF = PythonUDTF -@typing_extensions.final class CommonInlineUserDefinedDataSource(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3752,7 +3687,6 @@ class CommonInlineUserDefinedDataSource(google.protobuf.message.Message): global___CommonInlineUserDefinedDataSource = CommonInlineUserDefinedDataSource -@typing_extensions.final class PythonDataSource(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3775,7 +3709,6 @@ class PythonDataSource(google.protobuf.message.Message): global___PythonDataSource = PythonDataSource -@typing_extensions.final class CollectMetrics(google.protobuf.message.Message): """Collect arbitrary (named) metrics from a dataset.""" @@ -3816,7 +3749,6 @@ class CollectMetrics(google.protobuf.message.Message): global___CollectMetrics = CollectMetrics -@typing_extensions.final class Parse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3827,7 +3759,7 @@ class Parse(google.protobuf.message.Message): class _ParseFormatEnumTypeWrapper( google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[Parse._ParseFormat.ValueType], builtins.type, - ): + ): # noqa: F821 DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor PARSE_FORMAT_UNSPECIFIED: Parse._ParseFormat.ValueType # 0 PARSE_FORMAT_CSV: Parse._ParseFormat.ValueType # 1 @@ -3838,7 +3770,6 @@ class Parse(google.protobuf.message.Message): PARSE_FORMAT_CSV: Parse.ParseFormat.ValueType # 1 PARSE_FORMAT_JSON: Parse.ParseFormat.ValueType # 2 - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3906,7 +3837,6 @@ class Parse(google.protobuf.message.Message): global___Parse = Parse -@typing_extensions.final class AsOfJoin(google.protobuf.message.Message): """Relation of type [[AsOfJoin]]. diff --git a/python/pyspark/sql/connect/proto/types_pb2.pyi b/python/pyspark/sql/connect/proto/types_pb2.pyi index 8276c9d24918..e6b34d3485c2 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.pyi +++ b/python/pyspark/sql/connect/proto/types_pb2.pyi @@ -48,7 +48,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class DataType(google.protobuf.message.Message): """This message describes the logical [[DataType]] of something. It does not carry the value itself but only describes it. @@ -56,7 +55,6 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Boolean(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -74,7 +72,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Byte(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -92,7 +89,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Short(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -110,7 +106,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Integer(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -128,7 +123,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Long(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -146,7 +140,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Float(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -164,7 +157,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Double(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -182,7 +174,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class String(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -206,7 +197,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Binary(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -224,7 +214,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class NULL(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -242,7 +231,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Timestamp(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -260,7 +248,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Date(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -278,7 +265,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class TimestampNTZ(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -296,7 +282,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class CalendarInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -314,7 +299,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class YearMonthInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -368,7 +352,6 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_field", b"_start_field"] ) -> typing_extensions.Literal["start_field"] | None: ... - @typing_extensions.final class DayTimeInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -422,7 +405,6 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_field", b"_start_field"] ) -> typing_extensions.Literal["start_field"] | None: ... - @typing_extensions.final class Char(google.protobuf.message.Message): """Start compound types.""" @@ -445,7 +427,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class VarChar(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -466,7 +447,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Decimal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -520,7 +500,6 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_scale", b"_scale"] ) -> typing_extensions.Literal["scale"] | None: ... - @typing_extensions.final class StructField(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -566,7 +545,6 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_metadata", b"_metadata"] ) -> typing_extensions.Literal["metadata"] | None: ... - @typing_extensions.final class Struct(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -592,7 +570,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Array(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -625,7 +602,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Map(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -667,7 +643,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Variant(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -685,7 +660,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class UDT(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -765,7 +739,6 @@ class DataType(google.protobuf.message.Message): ], ) -> typing_extensions.Literal["serialized_python_class"] | None: ... - @typing_extensions.final class Unparsed(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor From b7d8cef9b8ee7af24536e140111f41c7c62bdd89 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Tue, 16 Apr 2024 18:52:14 +0200 Subject: [PATCH 16/20] address comments --- .../scala/org/apache/spark/sql/Dataset.scala | 9 ++---- .../org/apache/spark/sql/Observation.scala | 29 +---------------- .../org/apache/spark/sql/SparkSession.scala | 24 +++++++++++++- .../apache/spark/sql/ClientE2ETestSuite.scala | 20 ++++++++---- .../CheckConnectJvmClientCompatibility.scala | 2 -- .../sql/connect/client/SparkResult.scala | 32 +++++++++++-------- .../common/LiteralValueProtoConverter.scala | 2 +- 7 files changed, 60 insertions(+), 58 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 4eda2699f051..ea55ee93c1ea 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -3354,13 +3354,13 @@ class Dataset[T] private[sql] ( * function. * * A user can retrieve the metrics by calling - * `org.apache.spark.sql.Dataset.collectObservations()`. + * `org.apache.spark.sql.Dataset.collectResult().getObservedMetrics`. * * {{{ * // Observe row count (rows) and highest id (maxid) in the Dataset while writing it * val observed_ds = ds.observe("my_metrics", count(lit(1)).as("rows"), max($"id").as("maxid")) * observed_ds.write.parquet("ds.parquet") - * val metrics = observed_ds.collectObservations() + * val metrics = observed_ds.collectResult().getObservedMetrics * }}} * * @group typedrel @@ -3401,7 +3401,7 @@ class Dataset[T] private[sql] ( @scala.annotation.varargs def observe(observation: Observation, expr: Column, exprs: Column*): Dataset[T] = { val df = observe(observation.name, expr, exprs: _*) - observation.register(sparkSession, df.getPlanId.get) + sparkSession.registerObservation(df.getPlanId.get, observation) df } @@ -3463,9 +3463,6 @@ class Dataset[T] private[sql] ( def collectResult(): SparkResult[T] = sparkSession.execute(plan, agnosticEncoder) - def collectObservations(): Map[String, Map[String, Any]] = - collectResult().getObservedMetrics - private[sql] def withResult[E](f: SparkResult[T] => E): E = { val result = collectResult() try f(result) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala index df6e4af06b15..75629b6000f9 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Observation.scala @@ -25,38 +25,11 @@ class Observation(name: String) extends ObservationBase(name) { * Create an Observation instance without providing a name. This generates a random name. */ def this() = this(UUID.randomUUID().toString) - - @volatile private var planId: Option[(SparkSession, Long)] = None - - private[sql] def register(sparkSession: SparkSession, planId: Long): Unit = { - // makes this class thread-safe: - // only the first thread entering this block can set sparkSession - // all other threads will see the exception, as it is only allowed to do this once - synchronized { - if (this.planId.isDefined) { - throw new IllegalArgumentException("An Observation can be used with a Dataset only once") - } - this.planId = Some((sparkSession, planId)) - } - - sparkSession.observationRegistry.put(planId, this) - } - - private def unregister(): Unit = { - this.planId.map { case (sparkSession, planId) => - sparkSession.observationRegistry.remove(planId) - } - } - - override private[spark] def setMetricsAndNotify(metrics: Option[Map[String, Any]]): Boolean = { - this.unregister() - super.setMetricsAndNotify(metrics) - } } /** * (Scala-specific) Create instances of Observation via Scala `apply`. - * @since 3.3.0 + * @since 4.0.0 */ object Observation { diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 1904cbd219db..c9cca30d255b 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -555,7 +555,7 @@ class SparkSession private[sql] ( private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { val value = client.execute(plan) - new SparkResult(value, allocator, encoder, timeZoneId, Some(this.observationRegistry.toMap)) + new SparkResult(value, allocator, encoder, timeZoneId, Some(setMetricsAndUnregisterObservation)) } private[sql] def execute(f: proto.Relation.Builder => Unit): Unit = { @@ -818,6 +818,28 @@ class SparkSession private[sql] ( * Set to false to prevent client.releaseSession on close() (testing only) */ private[sql] var releaseSessionOnClose = true + + private[sql] def registerObservation(planId: Long, observation: Observation): Unit = { + // makes this class thread-safe: + // only the first thread entering this block can set sparkSession + // all other threads will see the exception, as it is only allowed to do this once + observation.synchronized { + if (observationRegistry.contains(planId)) { + throw new IllegalArgumentException("An Observation can be used with a Dataset only once") + } + observationRegistry.put(planId, observation) + } + } + + private[sql] def setMetricsAndUnregisterObservation( + planId: Long, + metrics: Option[Map[String, Any]]): Unit = { + observationRegistry.get(planId).map { observation => + if (observation.setMetricsAndNotify(metrics)) { + observationRegistry.remove(planId) + } + } + } } // The minimal builder needed to create a spark session. diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index 9d5f4aa4ba94..73a2f6d4f88e 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -1521,12 +1521,20 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM val observedDf = df.observe(ob1, min("id"), avg("id"), max("id")) val observedObservedDf = observedDf.observe("ob2", min("extra"), avg("extra"), max("extra")) - val ob1Metrics = Map("ob1" -> Map("min(id)" -> 0, "avg(id)" -> 49, "max(id)" -> 98)) - val ob2Metrics = Map("ob2" -> Map("min(extra)" -> -1, "avg(extra)" -> 48, "max(extra)" -> 97)) - - assert(df.collectObservations() === Map.empty) - assert(observedDf.collectObservations() === ob1Metrics) - assert(observedObservedDf.collectObservations() === ob1Metrics ++ ob2Metrics) + val ob1Schema = new StructType() + .add("min(id)", LongType) + .add("avg(id)", DoubleType) + .add("max(id)", LongType) + val ob2Schema = new StructType() + .add("min(extra)", LongType) + .add("avg(extra)", DoubleType) + .add("max(extra)", LongType) + val ob1Metrics = Map("ob1" -> new GenericRowWithSchema(Array(0, 49, 98), ob1Schema)) + val ob2Metrics = Map("ob2" -> new GenericRowWithSchema(Array(-1, 48, 97), ob2Schema)) + + assert(df.collectResult().getObservedMetrics === Map.empty) + assert(observedDf.collectResult().getObservedMetrics === ob1Metrics) + assert(observedObservedDf.collectResult().getObservedMetrics === ob1Metrics ++ ob2Metrics) } test("Observation.get is blocked until the query is finished") { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index 6fc5fc2ece37..5793581f2da2 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -360,8 +360,6 @@ object CheckConnectJvmClientCompatibility { ), // developer API ProblemFilters.exclude[DirectMissingMethodProblem]( "org.apache.spark.sql.Dataset.collectResult"), - ProblemFilters.exclude[DirectMissingMethodProblem]( - "org.apache.spark.sql.Dataset.collectObservations"), // RuntimeConfig ProblemFilters.exclude[MissingTypesProblem]( diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index 9e389f422395..2c62d6d7eaf1 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -28,9 +28,10 @@ import org.apache.arrow.vector.types.pojo import org.apache.spark.connect.proto import org.apache.spark.connect.proto.ExecutePlanResponse.ObservedMetrics -import org.apache.spark.sql.ObservationBase +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, UnboundRowEncoder} +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.connect.client.arrow.{AbstractMessageIterator, ArrowDeserializingIterator, ConcatenatingArrowStreamReader, MessageIterator} import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter} import org.apache.spark.sql.types.{DataType, StructType} @@ -41,7 +42,7 @@ private[sql] class SparkResult[T]( allocator: BufferAllocator, encoder: AgnosticEncoder[T], timeZoneId: String, - observationsOpt: Option[Map[Long, ObservationBase]] = None) + setObservationMetricsOpt: Option[(Long, Option[Map[String, Any]]) => Unit] = None) extends AutoCloseable { self => case class StageInfo( @@ -82,7 +83,7 @@ private[sql] class SparkResult[T]( private[this] var arrowSchema: pojo.Schema = _ private[this] var nextResultIndex: Int = 0 private val resultMap = mutable.Map.empty[Int, (Long, Seq[ArrowMessage])] - private val observedMetrics = mutable.Map.empty[String, Map[String, Any]] + private val observedMetrics = mutable.Map.empty[String, Row] private val cleanable = SparkResult.cleaner.register(this, new SparkResultCloseable(resultMap, responses)) @@ -206,23 +207,26 @@ private[sql] class SparkResult[T]( } private def processObservedMetrics( - metrics: java.util.List[ObservedMetrics]): Iterable[(String, Map[String, Any])] = { - val processed = mutable.ListBuffer.empty[(String, Map[String, Any])] - metrics.forEach { metric => + metrics: java.util.List[ObservedMetrics]): Iterable[(String, Row)] = { + metrics.asScala.map { metric => assert(metric.getKeysCount == metric.getValuesCount) - val kv = (0 until metric.getKeysCount).map { i => + var schema = new StructType() + val keys = mutable.ListBuffer.empty[String] + val values = mutable.ListBuffer.empty[Any] + (0 until metric.getKeysCount).map { i => val key = metric.getKeys(i) val value = LiteralValueProtoConverter.toCatalystValue(metric.getValues(i)) - key -> value - }.toMap - processed += metric.getName -> kv + schema = schema.add(key, LiteralValueProtoConverter.toDataType(value.getClass)) + keys += key + values += value + } // If the metrics is registered by an Observation object, attach them and unblock any // blocked thread. - observationsOpt.map { observations => - observations.get(metric.getPlanId).map(_.setMetricsAndNotify(Some(kv))) + setObservationMetricsOpt.foreach { setObservationMetrics => + setObservationMetrics(metric.getPlanId, Some(keys.zip(values).toMap)) } + metric.getName -> new GenericRowWithSchema(values.toArray, schema) } - processed } /** @@ -278,7 +282,7 @@ private[sql] class SparkResult[T]( /** * Returns all observed metrics in the result. */ - def getObservedMetrics: Map[String, Map[String, Any]] = { + def getObservedMetrics: Map[String, Row] = { // We need to process all responses to get all metrics. processResponses() observedMetrics.toMap diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala index ce42cc797bf3..1f3496fa8984 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala @@ -204,7 +204,7 @@ object LiteralValueProtoConverter { def toLiteralProto(literal: Any, dataType: DataType): proto.Expression.Literal = toLiteralProtoBuilder(literal, dataType).build() - private def toDataType(clz: Class[_]): DataType = clz match { + private[sql] def toDataType(clz: Class[_]): DataType = clz match { // primitive types case JShort.TYPE => ShortType case JInteger.TYPE => IntegerType From 3c0cf22bc8f9b6a7f5ae8fc09124147c7fb17fdf Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Wed, 17 Apr 2024 11:25:49 +0200 Subject: [PATCH 17/20] fmt --- .../src/main/scala/org/apache/spark/sql/SparkSession.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index c9cca30d255b..d7af3bfb36df 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -555,7 +555,12 @@ class SparkSession private[sql] ( private[sql] def execute[T](plan: proto.Plan, encoder: AgnosticEncoder[T]): SparkResult[T] = { val value = client.execute(plan) - new SparkResult(value, allocator, encoder, timeZoneId, Some(setMetricsAndUnregisterObservation)) + new SparkResult( + value, + allocator, + encoder, + timeZoneId, + Some(setMetricsAndUnregisterObservation)) } private[sql] def execute(f: proto.Relation.Builder => Unit): Unit = { From 4e1772f29ed18249f03400a04ebfff3dac0f675b Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Wed, 17 Apr 2024 16:44:53 +0200 Subject: [PATCH 18/20] fmt --- .../scala/org/apache/spark/sql/Dataset.scala | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index ea55ee93c1ea..37f770319b69 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -3339,14 +3339,11 @@ class Dataset[T] private[sql] ( /** * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset - * that returns the same result as the input, with the following guarantees: - *
    - *
  • It will compute the defined aggregates (metrics) on all the data that is flowing through - * the Dataset at that point.
  • - *
  • It will report the value of the defined aggregate columns as soon as we reach a - * completion point. A completion point is currently defined as the end of a query.
  • - *
- * Please note that continuous execution is currently not supported. + * that returns the same result as the input, with the following guarantees:
  • It will + * compute the defined aggregates (metrics) on all the data that is flowing through the Dataset + * at that point.
  • It will report the value of the defined aggregate columns as soon as + * we reach a completion point. A completion point is currently defined as the end of a + * query.
Please note that continuous execution is currently not supported. * * The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or * more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that @@ -3377,11 +3374,10 @@ class Dataset[T] private[sql] ( } /** - * Observe (named) metrics through an `org.apache.spark.sql.Observation` instance. - * This is equivalent to calling `observe(String, Column, Column*)` but does not require to - * collect all results before returning the metrics - the metrics are filled during iterating - * the results, as soon as they are available. - * This method does not support streaming datasets. + * Observe (named) metrics through an `org.apache.spark.sql.Observation` instance. This is + * equivalent to calling `observe(String, Column, Column*)` but does not require to collect all + * results before returning the metrics - the metrics are filled during iterating the results, + * as soon as they are available. This method does not support streaming datasets. * * A user can retrieve the metrics by accessing `org.apache.spark.sql.Observation.get`. * @@ -3393,7 +3389,8 @@ class Dataset[T] private[sql] ( * val metrics = observation.get * }}} * - * @throws IllegalArgumentException If this is a streaming Dataset (this.isStreaming == true) + * @throws IllegalArgumentException + * If this is a streaming Dataset (this.isStreaming == true) * * @group typedrel * @since 4.0.0 From 1ddbe92fb79e0602c0b67b210f97e8c534e37ace Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Fri, 19 Apr 2024 14:10:13 +0200 Subject: [PATCH 19/20] address comments --- .../org/apache/spark/sql/SparkSession.scala | 25 ++++++++----------- .../sql/connect/client/SparkResult.scala | 4 +-- 2 files changed, 12 insertions(+), 17 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index d7af3bfb36df..5a5b2c760b6f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql import java.io.Closeable import java.net.URI +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.TimeUnit._ import java.util.concurrent.atomic.{AtomicLong, AtomicReference} -import scala.collection.mutable import scala.jdk.CollectionConverters._ import scala.reflect.runtime.universe.TypeTag @@ -81,7 +81,7 @@ class SparkSession private[sql] ( client.analyze(proto.AnalyzePlanRequest.AnalyzeCase.SPARK_VERSION).getSparkVersion.getVersion } - private[sql] val observationRegistry = mutable.Map.empty[Long, Observation] + private[sql] val observationRegistry = new ConcurrentHashMap[Long, Observation]() /** * Runtime configuration interface for Spark. @@ -825,24 +825,19 @@ class SparkSession private[sql] ( private[sql] var releaseSessionOnClose = true private[sql] def registerObservation(planId: Long, observation: Observation): Unit = { - // makes this class thread-safe: - // only the first thread entering this block can set sparkSession - // all other threads will see the exception, as it is only allowed to do this once - observation.synchronized { - if (observationRegistry.contains(planId)) { - throw new IllegalArgumentException("An Observation can be used with a Dataset only once") - } - observationRegistry.put(planId, observation) + if (observationRegistry.containsKey(planId)) { + throw new IllegalArgumentException("An Observation can be used with a Dataset only once") } + observationRegistry.put(planId, observation) } private[sql] def setMetricsAndUnregisterObservation( planId: Long, - metrics: Option[Map[String, Any]]): Unit = { - observationRegistry.get(planId).map { observation => - if (observation.setMetricsAndNotify(metrics)) { - observationRegistry.remove(planId) - } + metrics: Map[String, Any]): Unit = { + if (observationRegistry.containsKey(planId)) { + val observation = observationRegistry.get(planId) + observation.setMetricsAndNotify(Some(metrics)) + observationRegistry.remove(planId) } } } diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala index 2c62d6d7eaf1..0905ee76c3f3 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala @@ -42,7 +42,7 @@ private[sql] class SparkResult[T]( allocator: BufferAllocator, encoder: AgnosticEncoder[T], timeZoneId: String, - setObservationMetricsOpt: Option[(Long, Option[Map[String, Any]]) => Unit] = None) + setObservationMetricsOpt: Option[(Long, Map[String, Any]) => Unit] = None) extends AutoCloseable { self => case class StageInfo( @@ -223,7 +223,7 @@ private[sql] class SparkResult[T]( // If the metrics is registered by an Observation object, attach them and unblock any // blocked thread. setObservationMetricsOpt.foreach { setObservationMetrics => - setObservationMetrics(metric.getPlanId, Some(keys.zip(values).toMap)) + setObservationMetrics(metric.getPlanId, keys.zip(values).toMap) } metric.getName -> new GenericRowWithSchema(values.toArray, schema) } From 294ec853df885d7b052ce3c04c8297fa3730f200 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Tue, 30 Apr 2024 11:08:02 +0200 Subject: [PATCH 20/20] . --- .../main/scala/org/apache/spark/sql/SparkSession.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 5a5b2c760b6f..fbb1738c44b2 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -825,19 +825,17 @@ class SparkSession private[sql] ( private[sql] var releaseSessionOnClose = true private[sql] def registerObservation(planId: Long, observation: Observation): Unit = { - if (observationRegistry.containsKey(planId)) { + if (observationRegistry.putIfAbsent(planId, observation) != null) { throw new IllegalArgumentException("An Observation can be used with a Dataset only once") } - observationRegistry.put(planId, observation) } private[sql] def setMetricsAndUnregisterObservation( planId: Long, metrics: Map[String, Any]): Unit = { - if (observationRegistry.containsKey(planId)) { - val observation = observationRegistry.get(planId) - observation.setMetricsAndNotify(Some(metrics)) - observationRegistry.remove(planId) + val observationOrNull = observationRegistry.remove(planId) + if (observationOrNull != null) { + observationOrNull.setMetricsAndNotify(Some(metrics)) } } }