(spark) branch master updated: [SPARK-45827] Disallow partitioning on Variant column

2024-01-19 Thread wenchen
This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 689ab0ee8ca8 [SPARK-45827] Disallow partitioning on Variant column
689ab0ee8ca8 is described below

commit 689ab0ee8ca87286db6167f672348e73116b9186
Author: cashmand 
AuthorDate: Sat Jan 20 10:45:51 2024 +0800

[SPARK-45827] Disallow partitioning on Variant column

### What changes were proposed in this pull request?

Follow-up to https://github.com/apache/spark/pull/43984: we should not 
allow partitioning on VariantType. Even though it is is an atomic type, it 
represents a nested semi-structured value, so not partitioning is consistent 
with our decision to not allow partitioning on nested types. Also, for now at 
least, it is not even comparable, so attempting to partition fails with a 
confusing codegen error about a method named `compare` not being declared.

### Why are the changes needed?

Improves error message when attempting to partition on Variant, and 
explicitly forbids a case that we do not intend to support.

### Does this PR introduce _any_ user-facing change?

Improved error message if a user tries to partition on Variant.

### How was this patch tested?

Added unit test, which fails without the code change.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44742 from cashmand/SPARK-45827-no-partitioning.

Authored-by: cashmand 
Signed-off-by: Wenchen Fan 
---
 .../execution/datasources/PartitioningUtils.scala  |  2 +-
 .../spark/sql/execution/datasources/rules.scala|  5 +-
 .../scala/org/apache/spark/sql/VariantSuite.scala  | 56 ++
 3 files changed, 60 insertions(+), 3 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
index 9905e9af9b0b..555099da221e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala
@@ -564,7 +564,7 @@ object PartitioningUtils extends SQLConfHelper {
 
 partitionColumnsSchema(schema, partitionColumns).foreach {
   field => field.dataType match {
-case _: AtomicType => // OK
+case a: AtomicType if !a.isInstanceOf[VariantType] => // OK
 case _ => throw 
QueryCompilationErrors.cannotUseDataTypeForPartitionColumnError(field)
   }
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
index c58815b6978e..bb2bad7a6867 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
@@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.command.DDLUtils
 import org.apache.spark.sql.execution.datasources.{CreateTable => 
CreateTableV1}
 import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2
 import org.apache.spark.sql.sources.InsertableRelation
-import org.apache.spark.sql.types.{AtomicType, StructType}
+import org.apache.spark.sql.types.{AtomicType, StructType, VariantType}
 import org.apache.spark.sql.util.PartitioningUtils.normalizePartitionSpec
 import org.apache.spark.sql.util.SchemaUtils
 import org.apache.spark.util.ArrayImplicits._
@@ -330,7 +330,8 @@ case class PreprocessTableCreation(catalog: SessionCatalog) 
extends Rule[Logical
 }
 
 schema.filter(f => 
normalizedPartitionCols.contains(f.name)).map(_.dataType).foreach {
-  case _: AtomicType => // OK
+  // VariantType values are not comparable, so can't be used as partition 
columns.
+  case a: AtomicType if !a.isInstanceOf[VariantType] => // OK
   case other => failAnalysis(s"Cannot use ${other.catalogString} for 
partition column")
 }
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala
index 98d106f05f0c..af37445c1323 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantSuite.scala
@@ -138,4 +138,60 @@ class VariantSuite extends QueryTest with 
SharedSparkSession {
   }
 }
   }
+
+  test("write partitioned file") {
+def verifyResult(df: DataFrame): Unit = {
+  val result = df.selectExpr("v").collect()
+.map(_.get(0).asInstanceOf[VariantVal].toString)
+.sorted
+.toSeq
+  val expected = (1 until 10).map(id => "1" * id)
+  assert(result == expected)
+}
+
+

(spark) branch master updated: [SPARK-46731][SS] Manage state store provider instance by state data source - reader

2024-01-19 Thread kabhwan
This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 56730f6390a1 [SPARK-46731][SS] Manage state store provider instance by 
state data source - reader
56730f6390a1 is described below

commit 56730f6390a19aeada75b866e64115a957212877
Author: Jungtaek Lim 
AuthorDate: Sat Jan 20 08:12:02 2024 +0900

[SPARK-46731][SS] Manage state store provider instance by state data source 
- reader

### What changes were proposed in this pull request?

This PR proposes to change state data source - reader part to manage state 
store provider instance by itself.

### Why are the changes needed?

Currently, state data source initializes state store instance via 
StateStore.get() which also initializes state store provider instance and 
registers the provider instance to the coordinator. This involves unnecessary 
overheads e.g. maintenance task could be triggered for this provider.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing UTs.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44751 from HeartSaVioR/SPARK-46731.

Authored-by: Jungtaek Lim 
Signed-off-by: Jungtaek Lim 
---
 .../v2/state/StatePartitionReader.scala| 16 +++--
 .../StreamStreamJoinStatePartitionReader.scala |  3 ++-
 .../state/SymmetricHashJoinStateManager.scala  | 28 ++
 3 files changed, 35 insertions(+), 12 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
index ef8d7bf628bf..b79079aca56e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, 
UnsafeRow}
 import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, 
PartitionReaderFactory}
 import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil
-import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, 
StateStore, StateStoreConf, StateStoreId, StateStoreProviderId}
+import org.apache.spark.sql.execution.streaming.state.{ReadStateStore, 
StateStoreConf, StateStoreId, StateStoreProvider, StateStoreProviderId}
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.util.SerializableConfiguration
 
@@ -53,15 +53,13 @@ class StatePartitionReader(
   private val keySchema = SchemaUtil.getSchemaAsDataType(schema, 
"key").asInstanceOf[StructType]
   private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, 
"value").asInstanceOf[StructType]
 
-  private lazy val store: ReadStateStore = {
+  private lazy val provider: StateStoreProvider = {
 val stateStoreId = 
StateStoreId(partition.sourceOptions.stateCheckpointLocation.toString,
   partition.sourceOptions.operatorId, partition.partition, 
partition.sourceOptions.storeName)
 val stateStoreProviderId = StateStoreProviderId(stateStoreId, 
partition.queryId)
-
 val allStateStoreMetadata = new StateMetadataPartitionReader(
   partition.sourceOptions.stateCheckpointLocation.getParent.toString, 
hadoopConf)
   .stateMetadata.toArray
-
 val stateStoreMetadata = allStateStoreMetadata.filter { entry =>
   entry.operatorId == partition.sourceOptions.operatorId &&
 entry.stateStoreName == partition.sourceOptions.storeName
@@ -78,9 +76,12 @@ class StatePartitionReader(
   stateStoreMetadata.head.numColsPrefixKey
 }
 
-StateStore.getReadOnly(stateStoreProviderId, keySchema, valueSchema,
-  numColsPrefixKey = numColsPrefixKey, version = 
partition.sourceOptions.batchId + 1,
-  storeConf = storeConf, hadoopConf = hadoopConf.value)
+StateStoreProvider.createAndInit(
+  stateStoreProviderId, keySchema, valueSchema, numColsPrefixKey, 
storeConf, hadoopConf.value)
+  }
+
+  private lazy val store: ReadStateStore = {
+provider.getReadStore(partition.sourceOptions.batchId + 1)
   }
 
   private lazy val iter: Iterator[InternalRow] = {
@@ -104,6 +105,7 @@ class StatePartitionReader(
   override def close(): Unit = {
 current = null
 store.abort()
+provider.close()
   }
 
   private def unifyStateRowPair(pair: (UnsafeRow, UnsafeRow)): InternalRow = {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala
 

(spark) branch master updated: [SPARK-46780][K8S][TESTS] Improve SBT K8s IT to skip R image build step if not needed

2024-01-19 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 3d400d333050 [SPARK-46780][K8S][TESTS] Improve SBT K8s IT to skip R 
image build step if not needed
3d400d333050 is described below

commit 3d400d333050f25e729db15817cff6940364ccce
Author: Dongjoon Hyun 
AuthorDate: Fri Jan 19 15:00:37 2024 -0800

[SPARK-46780][K8S][TESTS] Improve SBT K8s IT to skip R image build step if 
not needed

### What changes were proposed in this pull request?

This PR aims to allow the users to skip R image build step in SBT if R 
tests are excluded.

### Why are the changes needed?

`setup-integration-test-env.sh` has this feature already and has been used 
in Maven K8s IT.

https://github.com/apache/spark/blob/39f8e1a5953b5897f893151d24dc585a80c0c8a0/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh#L119-L123

This PR aims to improve SBT logic to match the behavior by skipping SparkR 
test if it's not needed.

### Does this PR introduce _any_ user-facing change?

No. This is a developer feature.

### How was this patch tested?

Manually tests.

**COMMAND**
```
$ build/sbt \
-Pkubernetes \
-Pkubernetes-integration-tests \
-Dtest.exclude.tags=minikube,local,r \
-Dspark.kubernetes.test.deployMode=docker-desktop \
"kubernetes-integration-tests/test"
```

**BEFORE**
```
$ docker images | grep spark
kubespark/spark-r dev   
 443f71dfbf87   17 seconds ago   1.53GB
kubespark/spark-pydev   
 82e3d4a859f6   About a minute ago   1.07GB
kubespark/spark   dev   
 2eefd2785313   2 minutes ago716MB
```

**AFTER**
```
$ docker images | grep spark
kubespark/spark-pydev   
 155ed5f5dd95   10 seconds ago   1.07GB
kubespark/spark   dev   
 3edf056c5b33   About a minute ago   716MB
```

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44807 from dongjoon-hyun/SPARK-46780.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 project/SparkBuild.scala | 6 +-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 7d399a735888..3123f931cd88 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -995,8 +995,12 @@ object KubernetesIntegrationTests {
 
s"$sparkHome/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile")
 val pyDockerFile = 
sys.props.getOrElse("spark.kubernetes.test.pyDockerFile",
 s"$bindingsDir/python/Dockerfile")
-val rDockerFile = 
sys.props.getOrElse("spark.kubernetes.test.rDockerFile",
+var rDockerFile = 
sys.props.getOrElse("spark.kubernetes.test.rDockerFile",
 s"$bindingsDir/R/Dockerfile")
+val excludeTags = sys.props.getOrElse("test.exclude.tags", 
"").split(",")
+if (excludeTags.exists(_.equalsIgnoreCase("r"))) {
+  rDockerFile = ""
+}
 val extraOptions = if (javaImageTag.isDefined) {
   Seq("-b", s"java_image_tag=$javaImageTag")
 } else {


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated (3d395a6b874b -> 39f8e1a5953b)

2024-01-19 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


from 3d395a6b874b [SPARK-46774][SQL][AVRO] Use 
mapreduce.output.fileoutputformat.compress instead of deprecated 
mapred.output.compress in Avro write jobs
 add 39f8e1a5953b [SPARK-46773][BUILD][CONNECT] Change to use include-list 
to `generate assemblyExcludedJars` for the connect server module

No new revisions were added by this update.

Summary of changes:
 project/SparkBuild.scala | 18 --
 1 file changed, 12 insertions(+), 6 deletions(-)


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-46774][SQL][AVRO] Use mapreduce.output.fileoutputformat.compress instead of deprecated mapred.output.compress in Avro write jobs

2024-01-19 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 3d395a6b874b [SPARK-46774][SQL][AVRO] Use 
mapreduce.output.fileoutputformat.compress instead of deprecated 
mapred.output.compress in Avro write jobs
3d395a6b874b is described below

commit 3d395a6b874bfd4609323551773c61d42fb60b8a
Author: Kent Yao 
AuthorDate: Fri Jan 19 10:22:13 2024 -0800

[SPARK-46774][SQL][AVRO] Use mapreduce.output.fileoutputformat.compress 
instead of deprecated mapred.output.compress in Avro write jobs

### What changes were proposed in this pull request?

According to 
[DeprecatedProperties](https://hadoop.apache.org/docs/r3.3.6/hadoop-project-dist/hadoop-common/DeprecatedProperties.html),
 `mapred.output.compress` is deprecated. So in this PR, we use 
`mapreduce.output.fileoutputformat.compress` .

### Why are the changes needed?

remove usage of deprecated Hadoop configurations

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

I tested locally by verifying the compressed output files before and after 
this change.

### Was this patch authored or co-authored using generative AI tooling?

no

Closes #44799 from yaooqinn/SPARK-46774.

Authored-by: Kent Yao 
Signed-off-by: Dongjoon Hyun 
---
 .../avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala 
b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
index f0b70f09aa55..05562c913b19 100644
--- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
+++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
@@ -107,9 +107,9 @@ private[sql] object AvroUtils extends Logging {
 val jobConf = job.getConfiguration
 AvroCompressionCodec.fromString(codecName) match {
   case UNCOMPRESSED =>
-jobConf.setBoolean("mapred.output.compress", false)
+jobConf.setBoolean("mapreduce.output.fileoutputformat.compress", 
false)
   case compressed =>
-jobConf.setBoolean("mapred.output.compress", true)
+jobConf.setBoolean("mapreduce.output.fileoutputformat.compress", 
true)
 jobConf.set(AvroJob.CONF_OUTPUT_CODEC, compressed.getCodecName)
 if (compressed.getSupportCompressionLevel) {
   val level = 
sqlConf.getConfString(s"spark.sql.avro.$codecName.level",


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-46766][SQL][AVRO] ZSTD Buffer Pool Support For AVRO datasource

2024-01-19 Thread yao
This is an automated email from the ASF dual-hosted git repository.

yao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c4af64ec0dbb [SPARK-46766][SQL][AVRO] ZSTD Buffer Pool Support For 
AVRO datasource
c4af64ec0dbb is described below

commit c4af64ec0dbb3cb3bda6debd38009bbe9844638c
Author: Kent Yao 
AuthorDate: Fri Jan 19 17:32:24 2024 +0800

[SPARK-46766][SQL][AVRO] ZSTD Buffer Pool Support For AVRO datasource

### What changes were proposed in this pull request?

This PR adds ZSTD Buffer Pool Support For AVRO datasource writing with zstd 
compression codec

### Why are the changes needed?

Enable a tuning technique for users

### Does this PR introduce _any_ user-facing change?

yes, add a new configuration

### How was this patch tested?

passing existing ci shall be sufficient

### Was this patch authored or co-authored using generative AI tooling?

no

Closes #44792 from yaooqinn/SPARK-46766.

Authored-by: Kent Yao 
Signed-off-by: Kent Yao 
---
 .../scala/org/apache/spark/sql/avro/AvroUtils.scala   | 19 +--
 docs/sql-data-sources-avro.md |  8 
 .../scala/org/apache/spark/sql/internal/SQLConf.scala |  6 ++
 3 files changed, 27 insertions(+), 6 deletions(-)

diff --git 
a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala 
b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
index d9c88e14d039..f0b70f09aa55 100644
--- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
+++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
@@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._
 import org.apache.avro.Schema
 import org.apache.avro.file.{DataFileReader, FileReader}
 import org.apache.avro.generic.{GenericDatumReader, GenericRecord}
-import org.apache.avro.mapred.FsInput
+import org.apache.avro.mapred.{AvroOutputFormat, FsInput}
 import org.apache.avro.mapreduce.AvroJob
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.FileStatus
@@ -104,18 +104,25 @@ private[sql] object AvroUtils extends Logging {
 
 parsedOptions.compression.toLowerCase(Locale.ROOT) match {
   case codecName if AvroCompressionCodec.values().exists(c => 
c.lowerCaseName() == codecName) =>
+val jobConf = job.getConfiguration
 AvroCompressionCodec.fromString(codecName) match {
   case UNCOMPRESSED =>
-job.getConfiguration.setBoolean("mapred.output.compress", false)
+jobConf.setBoolean("mapred.output.compress", false)
   case compressed =>
-job.getConfiguration.setBoolean("mapred.output.compress", true)
-job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, 
compressed.getCodecName)
+jobConf.setBoolean("mapred.output.compress", true)
+jobConf.set(AvroJob.CONF_OUTPUT_CODEC, compressed.getCodecName)
 if (compressed.getSupportCompressionLevel) {
   val level = 
sqlConf.getConfString(s"spark.sql.avro.$codecName.level",
 compressed.getDefaultCompressionLevel.toString)
   logInfo(s"Compressing Avro output using the $codecName codec at 
level $level")
-  val s = if (compressed == ZSTANDARD) "zstd" else codecName
-  job.getConfiguration.setInt(s"avro.mapred.$s.level", level.toInt)
+  val s = if (compressed == ZSTANDARD) {
+val bufferPoolEnabled = 
sqlConf.getConf(SQLConf.AVRO_ZSTANDARD_BUFFER_POOL_ENABLED)
+jobConf.setBoolean(AvroOutputFormat.ZSTD_BUFFERPOOL_KEY, 
bufferPoolEnabled)
+"zstd"
+  } else {
+codecName
+  }
+  jobConf.setInt(s"avro.mapred.$s.level", level.toInt)
 } else {
   logInfo(s"Compressing Avro output using the $codecName codec")
 }
diff --git a/docs/sql-data-sources-avro.md b/docs/sql-data-sources-avro.md
index e4b4963f7b5f..2172cb68fb98 100644
--- a/docs/sql-data-sources-avro.md
+++ b/docs/sql-data-sources-avro.md
@@ -380,6 +380,14 @@ Configuration of Avro can be done via `spark.conf.set` or 
by running `SET key=va
 
 4.0.0
   
+  
+spark.sql.avro.zstandard.bufferPool.enabled
+false
+
+  If true, enable buffer pool of ZSTD JNI library when writing of AVRO 
files.
+
+4.0.0
+  
   
 spark.sql.avro.datetimeRebaseModeInRead
 EXCEPTION
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 61c7b2457b11..3dd7cf884cbe 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ 

(spark) branch master updated: [SPARK-46768][BUILD] Upgrade Guava used by the connect module to 33.0-jre

2024-01-19 Thread yao
This is an automated email from the ASF dual-hosted git repository.

yao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new a09b726f880c [SPARK-46768][BUILD] Upgrade Guava used by the connect 
module to 33.0-jre
a09b726f880c is described below

commit a09b726f880c8369976a9c78f1dffce1969358ca
Author: yangjie01 
AuthorDate: Fri Jan 19 17:20:25 2024 +0800

[SPARK-46768][BUILD] Upgrade Guava used by the connect module to 33.0-jre

### What changes were proposed in this pull request?
This pr aims to upgrade Guava used by the `connect` module from 32.0.1-jre 
to 33.0-jre, at the same time, upgrade ·failureaccess·, which is used in 
conjunction with Guava, from version 1.01 to 1.02.

### Why are the changes needed?
The new version bring some changes as follows:
- net: Optimized InternetDomainName construction. 
(https://github.com/google/guava/commit/3a1d18fbefa10218988a0fbbb6e1fada012397bf,
 
https://github.com/google/guava/commit/eaa62eb09548a6f1b7a757e21d8852724b631cab)
- util.concurrent: Changed our implementations to avoid eagerly 
initializing loggers during class loading. This can help performance. 
(https://github.com/google/guava/commit/4fe1df56bd74e9eec8847bdb15c5be51f528e8c8)

The full release notes as follows:
- https://github.com/google/guava/releases/tag/v32.1.2
- https://github.com/google/guava/releases/tag/v32.1.3
- https://github.com/google/guava/releases/tag/v33.0.0

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass GitHub Actions

### Was this patch authored or co-authored using generative AI tooling?
No

Closes #44795 from LuciferYang/upgrade-connect-guava.

Authored-by: yangjie01 
Signed-off-by: Kent Yao 
---
 pom.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/pom.xml b/pom.xml
index 3eb8b0917bf5..e290273543c6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -291,8 +291,8 @@
 
true
 
 
-32.0.1-jre
-1.0.1
+33.0.0-jre
+1.0.2
 1.59.0
 1.1.3
 6.0.53


-
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org



(spark) branch master updated: [SPARK-46770][K8S][TESTS] Remove legacy `docker-for-desktop` logic

2024-01-19 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new a569fca2e9b4 [SPARK-46770][K8S][TESTS] Remove legacy 
`docker-for-desktop` logic
a569fca2e9b4 is described below

commit a569fca2e9b4b819494e23e0c63bfd69101f02e0
Author: Dongjoon Hyun 
AuthorDate: Fri Jan 19 00:22:25 2024 -0800

[SPARK-46770][K8S][TESTS] Remove legacy `docker-for-desktop` logic

### What changes were proposed in this pull request?

This PR aims to remove legacy `docker-for-desktop` logic in favor of 
`docker-desktop`.

### Why are the changes needed?

- Docker Desktop switched the underlying node name and context to 
`docker-desktop` in 2020.
  - https://github.com/docker/for-win/issues/5089#issuecomment-582752325
- Since Apache Spark 3.2.2, we have been hiding it from the documentation 
via SPARK-38272 and now we can delete it.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CIs and manually test with Docker Desktop.

```
$ build/sbt -Psparkr -Pkubernetes -Pkubernetes-integration-tests 
-Dtest.exclude.tags=minikube,local 
-Dspark.kubernetes.test.deployMode=docker-desktop 
"kubernetes-integration-tests/test"
...
[info] KubernetesSuite:
[info] - SPARK-42190: Run SparkPi with local[*] (12 seconds, 759 
milliseconds)
[info] - Run SparkPi with no resources (13 seconds, 747 milliseconds)
[info] - Run SparkPi with no resources & statefulset allocation (19 
seconds, 688 milliseconds)
[info] - Run SparkPi with a very long application name. (12 seconds, 436 
milliseconds)
[info] - Use SparkLauncher.NO_RESOURCE (17 seconds, 411 milliseconds)
[info] - Run SparkPi with a master URL without a scheme. (12 seconds, 352 
milliseconds)
[info] - Run SparkPi with an argument. (17 seconds, 481 milliseconds)
[info] - Run SparkPi with custom labels, annotations, and environment 
variables. (12 seconds, 375 milliseconds)
[info] - All pods have the same service account by default (17 seconds, 375 
milliseconds)
[info] - Run extraJVMOptions check on driver (9 seconds, 362 milliseconds)
[info] - SPARK-42474: Run extraJVMOptions JVM GC option check - G1GC (12 
seconds, 319 milliseconds)
[info] - SPARK-42474: Run extraJVMOptions JVM GC option check - Other GC (9 
seconds, 280 milliseconds
[info] - SPARK-42769: All executor pods have SPARK_DRIVER_POD_IP env 
variable (12 seconds, 404 milliseconds)
[info] - Verify logging configuration is picked from the provided 
SPARK_CONF_DIR/log4j2.properties (18 seconds, 198 milliseconds)
[info] - Run SparkPi with env and mount secrets. (19 seconds, 463 
milliseconds)
[info] - Run PySpark on simple pi.py example (18 seconds, 373 milliseconds)
[info] - Run PySpark to test a pyfiles example (14 seconds, 435 
milliseconds)
[info] - Run PySpark with memory customization (17 seconds, 334 
milliseconds)
[info] - Run in client mode. (5 seconds, 235 milliseconds)
[info] - Start pod creation from template (12 seconds, 447 milliseconds)
[info] - SPARK-38398: Schedule pod creation from template (17 seconds, 351 
milliseconds)
[info] - Test basic decommissioning (45 seconds, 365 milliseconds)
[info] - Test basic decommissioning with shuffle cleanup (49 seconds, 679 
milliseconds)
[info] - Test decommissioning with dynamic allocation & shuffle cleanups (2 
minutes, 52 seconds)
[info] - Test decommissioning timeouts (50 seconds, 379 milliseconds)
[info] - SPARK-37576: Rolling decommissioning (1 minute, 17 seconds)
[info] - Run SparkR on simple dataframe.R example (19 seconds, 453 
milliseconds)
[info] YuniKornSuite:
[info] Run completed in 14 minutes, 39 seconds.
[info] Total number of tests run: 27
[info] Suites: completed 2, aborted 0
[info] Tests: succeeded 27, failed 0, canceled 0, ignored 0, pending 0
[info] All tests passed.
[success] Total time: 1078 s (17:58), completed Jan 19, 2024, 12:12:23 AM
```

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44796 from dongjoon-hyun/SPARK-46770.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../kubernetes/integration-tests/scripts/setup-integration-test-env.sh  | 2 +-
 .../org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala  | 2 +-
 .../org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala | 1 -
 .../deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala | 2 +-
 4 files changed, 3 insertions(+), 4 deletions(-)

diff --git 
a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh