Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-49709][CONNECT][SQL] Support ConfigEntry in the RuntimeConfig interface #49062

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.util.matching.Regex

import org.apache.spark.SparkIllegalArgumentException
import org.apache.spark.network.util.{ByteUnit, JavaUtils}
import org.apache.spark.util.Utils
import org.apache.spark.util.SparkStringUtils

private object ConfigHelpers {

Expand All @@ -47,7 +47,7 @@ private object ConfigHelpers {
}

def stringToSeq[T](str: String, converter: String => T): Seq[T] = {
Utils.stringToSeq(str).map(converter)
SparkStringUtils.stringToSeq(str).map(converter)
}

def seqToString[T](v: Seq[T], stringConverter: T => String): String = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@ package org.apache.spark.internal.config

import java.util.{Map => JMap}

import org.apache.spark.SparkConf

/**
* A source of configuration values.
*/
Expand All @@ -47,18 +45,3 @@ private[spark] class MapProvider(conf: JMap[String, String]) extends ConfigProvi
override def get(key: String): Option[String] = Option(conf.get(key))

}

/**
* A config provider that only reads Spark config keys.
*/
private[spark] class SparkConfigProvider(conf: JMap[String, String]) extends ConfigProvider {

override def get(key: String): Option[String] = {
if (key.startsWith("spark.")) {
Option(conf.get(key)).orElse(SparkConf.getDeprecatedConfig(key, conf))
} else {
None
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.util

trait SparkStringUtils {
def stringToSeq(str: String): Seq[String] = {
import org.apache.spark.util.ArrayImplicits._
str.split(",").map(_.trim()).filter(_.nonEmpty).toImmutableArraySeq
}
}

object SparkStringUtils extends SparkStringUtils
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ package org.apache.spark.sql.internal

import org.apache.spark.connect.proto.{ConfigRequest, ConfigResponse, KeyValue}
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.{ConfigEntry, ConfigReader, OptionalConfigEntry}
import org.apache.spark.sql.RuntimeConfig
import org.apache.spark.sql.connect.client.SparkConnectClient

Expand All @@ -28,7 +29,7 @@ import org.apache.spark.sql.connect.client.SparkConnectClient
*/
class ConnectRuntimeConfig private[sql] (client: SparkConnectClient)
extends RuntimeConfig
with Logging {
with Logging { self =>

/** @inheritdoc */
def set(key: String, value: String): Unit = {
Expand All @@ -37,6 +38,13 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient)
}
}

/** @inheritdoc */
override private[sql] def set[T](entry: ConfigEntry[T], value: T): Unit = {
require(entry != null, "entry cannot be null")
require(value != null, s"value cannot be null for key: ${entry.key}")
set(entry.key, entry.stringConverter(value))
}

/** @inheritdoc */
@throws[NoSuchElementException]("if the key is not set and there is no default value")
def get(key: String): String = getOption(key).getOrElse {
Expand All @@ -45,11 +53,39 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient)

/** @inheritdoc */
def get(key: String, default: String): String = {
executeConfigRequestSingleValue { builder =>
builder.getGetWithDefaultBuilder.addPairsBuilder().setKey(key).setValue(default)
val kv = executeConfigRequestSinglePair { builder =>
val pairsBuilder = builder.getGetWithDefaultBuilder
.addPairsBuilder()
.setKey(key)
if (default != null) {
pairsBuilder.setValue(default)
}
}
if (kv.hasValue) {
kv.getValue
} else {
default
}
}

/** @inheritdoc */
override private[sql] def get[T](entry: ConfigEntry[T]): T = {
require(entry != null, "entry cannot be null")
entry.readFrom(reader)
}

/** @inheritdoc */
override private[sql] def get[T](entry: OptionalConfigEntry[T]): Option[T] = {
require(entry != null, "entry cannot be null")
entry.readFrom(reader)
}

/** @inheritdoc */
override private[sql] def get[T](entry: ConfigEntry[T], default: T): T = {
require(entry != null, "entry cannot be null")
Option(get(entry.key, null)).map(entry.valueConverter).getOrElse(default)
}

/** @inheritdoc */
def getAll: Map[String, String] = {
val response = executeConfigRequest { builder =>
Expand All @@ -65,11 +101,11 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient)

/** @inheritdoc */
def getOption(key: String): Option[String] = {
val pair = executeConfigRequestSinglePair { builder =>
val kv = executeConfigRequestSinglePair { builder =>
builder.getGetOptionBuilder.addKeys(key)
}
if (pair.hasValue) {
Option(pair.getValue)
if (kv.hasValue) {
Option(kv.getValue)
} else {
None
}
Expand All @@ -84,17 +120,11 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient)

/** @inheritdoc */
def isModifiable(key: String): Boolean = {
val modifiable = executeConfigRequestSingleValue { builder =>
val kv = executeConfigRequestSinglePair { builder =>
builder.getIsModifiableBuilder.addKeys(key)
}
java.lang.Boolean.valueOf(modifiable)
}

private def executeConfigRequestSingleValue(
f: ConfigRequest.Operation.Builder => Unit): String = {
val pair = executeConfigRequestSinglePair(f)
require(pair.hasValue, "The returned pair does not have a value set")
pair.getValue
require(kv.hasValue, "The returned pair does not have a value set")
java.lang.Boolean.valueOf(kv.getValue)
}

private def executeConfigRequestSinglePair(
Expand All @@ -113,4 +143,6 @@ class ConnectRuntimeConfig private[sql] (client: SparkConnectClient)
}
response
}

private val reader = new ConfigReader((key: String) => Option(self.get(key, null)))
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import org.scalatest.PrivateMethodTester

import org.apache.spark.{SparkArithmeticException, SparkException, SparkUpgradeException}
import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION}
import org.apache.spark.internal.config.ConfigBuilder
import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, TableAlreadyExistsException, TempTableAlreadyExistsException}
import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.StringEncoder
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
Expand Down Expand Up @@ -1006,8 +1007,12 @@ class ClientE2ETestSuite
test("RuntimeConfig") {
intercept[NoSuchElementException](spark.conf.get("foo.bar"))
assert(spark.conf.getOption("foo.bar").isEmpty)
assert(spark.conf.get("foo.bar", "nope") == "nope")
assert(spark.conf.get("foo.bar", null) == null)
spark.conf.set("foo.bar", value = true)
assert(spark.conf.getOption("foo.bar") === Option("true"))
assert(spark.conf.get("foo.bar", "nope") === "true")
assert(spark.conf.get("foo.bar", null) === "true")
spark.conf.set("foo.bar.numBaz", 100L)
assert(spark.conf.get("foo.bar.numBaz") === "100")
spark.conf.set("foo.bar.name", "donkey")
Expand All @@ -1020,6 +1025,24 @@ class ClientE2ETestSuite
assert(spark.conf.isModifiable("spark.sql.ansi.enabled"))
assert(!spark.conf.isModifiable("spark.sql.globalTempDatabase"))
intercept[Exception](spark.conf.set("spark.sql.globalTempDatabase", "/dev/null"))

val entry = ConfigBuilder("my.simple.conf").intConf.createOptional
intercept[NoSuchElementException](spark.conf.get(entry.key))
assert(spark.conf.get(entry).isEmpty)
assert(spark.conf.get(entry, Option(55)) === Option(55))
spark.conf.set(entry, Option(33))
assert(spark.conf.get(entry.key) === "33")
assert(spark.conf.get(entry) === Option(33))
assert(spark.conf.get(entry, Option(55)) === Option(33))

val entryWithDefault = ConfigBuilder("my.important.conf").intConf.createWithDefault(10)
intercept[NoSuchElementException](spark.conf.get(entryWithDefault.key))
assert(spark.conf.get(entryWithDefault) === 10)
assert(spark.conf.get(entryWithDefault, 11) === 11)
spark.conf.set(entryWithDefault, 12)
assert(spark.conf.get(entryWithDefault.key) === "12")
assert(spark.conf.get(entryWithDefault) === 12)
assert(spark.conf.get(entryWithDefault, 11) === 12)
}

test("SparkVersion") {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.internal.config

import java.util.{Map => JMap}

import org.apache.spark.SparkConf

/**
* A config provider that only reads Spark config keys.
*/
private[spark] class SparkConfigProvider(conf: JMap[String, String]) extends ConfigProvider {

override def get(key: String): Option[String] = {
if (key.startsWith("spark.")) {
Option(conf.get(key)).orElse(SparkConf.getDeprecatedConfig(key, conf))
} else {
None
}
}
}
7 changes: 2 additions & 5 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,8 @@ private[spark] object Utils
with SparkErrorUtils
with SparkFileUtils
with SparkSerDeUtils
with SparkStreamUtils {
with SparkStreamUtils
with SparkStringUtils {

private val sparkUncaughtExceptionHandler = new SparkUncaughtExceptionHandler
@volatile private var cachedLocalDir: String = ""
Expand Down Expand Up @@ -2799,10 +2800,6 @@ private[spark] object Utils
}
}

def stringToSeq(str: String): Seq[String] = {
str.split(",").map(_.trim()).filter(_.nonEmpty).toImmutableArraySeq
}

/**
* Create instances of extension classes.
*
Expand Down
25 changes: 25 additions & 0 deletions sql/api/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.spark.sql

import org.apache.spark.annotation.Stable
import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry}

/**
* Runtime configuration interface for Spark. To access this, use `SparkSession.conf`.
Expand Down Expand Up @@ -53,6 +54,11 @@ abstract class RuntimeConfig {
set(key, value.toString)
}

/**
* Sets the given Spark runtime configuration property.
*/
private[sql] def set[T](entry: ConfigEntry[T], value: T): Unit

/**
* Returns the value of Spark runtime configuration property for the given key. If the key is
* not set yet, return its default value if possible, otherwise `NoSuchElementException` will be
Expand All @@ -74,6 +80,25 @@ abstract class RuntimeConfig {
*/
def get(key: String, default: String): String

/**
* Returns the value of Spark runtime configuration property for the given key. If the key is
* not set yet, return `defaultValue` in [[ConfigEntry]].
*/
@throws[NoSuchElementException]("if the key is not set")
private[sql] def get[T](entry: ConfigEntry[T]): T

/**
* Returns the value of Spark runtime configuration property for the given key. If the key is
* not set yet, return None.
*/
private[sql] def get[T](entry: OptionalConfigEntry[T]): Option[T]

/**
* Returns the value of Spark runtime configuration property for the given key. If the key is
* not set yet, return the user given `default`.
*/
private[sql] def get[T](entry: ConfigEntry[T], default: T): T

/**
* Returns all properties set in this conf.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import scala.jdk.CollectionConverters._

import org.apache.spark.SPARK_DOC_ROOT
import org.apache.spark.annotation.Stable
import org.apache.spark.internal.config.{ConfigEntry, DEFAULT_PARALLELISM}
import org.apache.spark.internal.config.{ConfigEntry, DEFAULT_PARALLELISM, OptionalConfigEntry}
import org.apache.spark.sql.RuntimeConfig
import org.apache.spark.sql.errors.QueryCompilationErrors

Expand All @@ -41,6 +41,12 @@ class RuntimeConfigImpl private[sql](val sqlConf: SQLConf = new SQLConf) extends
sqlConf.setConfString(key, value)
}

/** @inheritdoc */
override private[sql] def set[T](entry: ConfigEntry[T], value: T): Unit = {
requireNonStaticConf(entry.key)
sqlConf.setConf(entry, value)
}

/** @inheritdoc */
@throws[NoSuchElementException]("if the key is not set and there is no default value")
def get(key: String): String = {
Expand All @@ -57,6 +63,18 @@ class RuntimeConfigImpl private[sql](val sqlConf: SQLConf = new SQLConf) extends
sqlConf.getAllConfs
}

/** @inheritdoc */
override private[sql] def get[T](entry: ConfigEntry[T]): T =
sqlConf.getConf(entry)

/** @inheritdoc */
override private[sql] def get[T](entry: OptionalConfigEntry[T]): Option[T] =
sqlConf.getConf(entry)

/** @inheritdoc */
override private[sql] def get[T](entry: ConfigEntry[T], default: T): T =
sqlConf.getConf(entry, default)

private[sql] def getAllAsJava: java.util.Map[String, String] = {
getAll.asJava
}
Expand Down
Loading
Loading