(spark) branch master updated: [SPARK-46389][CORE] Manually close the `RocksDB/LevelDB` instance when `checkVersion` throw Exception

2023-12-17 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 5e1b904ca54f [SPARK-46389][CORE] Manually close the `RocksDB/LevelDB` 
instance when `checkVersion` throw Exception
5e1b904ca54f is described below

commit 5e1b904ca54f8eddc5315933e43edc8bdd0d2982
Author: yangjie01 
AuthorDate: Sun Dec 17 13:22:13 2023 -0800

[SPARK-46389][CORE] Manually close the `RocksDB/LevelDB` instance when 
`checkVersion` throw Exception

### What changes were proposed in this pull request?
In the process of initializing the `DB` in 
`RocksDBProvider/LevelDBProvider`, there is a `checkVersion` step that may 
throw an exception. After the exception is thrown, the upper-level caller 
cannot hold the already opened `RockDB/LevelDB` instance, so it cannot perform 
resource cleanup, which poses a potential risk of handle leakage. So this PR 
manually closes the `RocksDB/LevelDB` instance when `checkVersion` throws an 
exception.

### Why are the changes needed?
Should close the `RocksDB/LevelDB` instance when `checkVersion` throw 
Exception

### 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 #44327 from LuciferYang/SPARK-46389.

Authored-by: yangjie01 
Signed-off-by: Dongjoon Hyun 
---
 .../main/java/org/apache/spark/network/util/LevelDBProvider.java   | 7 ++-
 .../main/java/org/apache/spark/network/util/RocksDBProvider.java   | 4 
 2 files changed, 10 insertions(+), 1 deletion(-)

diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java
 
b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java
index b27e3beb77ef..aa8be0c663bc 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/util/LevelDBProvider.java
@@ -80,7 +80,12 @@ public class LevelDBProvider {
 }
   }
   // if there is a version mismatch, we throw an exception, which means 
the service is unusable
-  checkVersion(tmpDb, version, mapper);
+  try {
+checkVersion(tmpDb, version, mapper);
+  } catch (IOException ioe) {
+tmpDb.close();
+throw ioe;
+  }
 }
 return tmpDb;
   }
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java
 
b/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java
index f1f702c44245..f3b7b48355a0 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/util/RocksDBProvider.java
@@ -100,7 +100,11 @@ public class RocksDBProvider {
   // is unusable
   checkVersion(tmpDb, version, mapper);
 } catch (RocksDBException e) {
+  tmpDb.close();
   throw new IOException(e.getMessage(), e);
+} catch (IOException ioe) {
+  tmpDb.close();
+  throw ioe;
 }
   }
   return tmpDb;


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



(spark) branch master updated: [SPARK-46376][SQL][TESTS] Simplify the code to generate the Spark tarball `filename` in the `HiveExternalCatalogVersionsSuite`

2023-12-17 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 0745bb507f36 [SPARK-46376][SQL][TESTS] Simplify the code to generate 
the Spark tarball `filename` in the `HiveExternalCatalogVersionsSuite`
0745bb507f36 is described below

commit 0745bb507f36b8201d49d886fc5da436274e8b85
Author: yangjie01 
AuthorDate: Sun Dec 17 13:20:51 2023 -0800

[SPARK-46376][SQL][TESTS] Simplify the code to generate the Spark tarball 
`filename` in the `HiveExternalCatalogVersionsSuite`

### What changes were proposed in this pull request?
This PR simplifies the code used to generate the Spark tarball `filename` 
in `HiveExternalCatalogVersionsSuite` because the minimum tested version is 
Spark 3.4.

### Why are the changes needed?
Simplify the code to generate the Spark tarball `filename`

### 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 #44307 from LuciferYang/SPARK-46376.

Authored-by: yangjie01 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/sql/hive/HiveExternalCatalogVersionsSuite.scala  | 10 ++
 1 file changed, 2 insertions(+), 8 deletions(-)

diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala
index 52f20595a10a..ee2e64bc1905 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala
@@ -40,8 +40,8 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType
 import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.tags.{ExtendedHiveTest, SlowHiveTest}
-import org.apache.spark.util.{Utils, VersionUtils}
 import org.apache.spark.util.ArrayImplicits._
+import org.apache.spark.util.Utils
 
 /**
  * Test HiveExternalCatalog backward compatibility.
@@ -95,13 +95,7 @@ class HiveExternalCatalogVersionsSuite extends 
SparkSubmitTestUtils {
   mirrors.distinct :+ "https://archive.apache.org/dist; :+ 
PROCESS_TABLES.releaseMirror
 logInfo(s"Trying to download Spark $version from $sites")
 for (site <- sites) {
-  val filename = VersionUtils.majorMinorPatchVersion(version) match {
-case Some((major, _, _)) if major > 3 => 
s"spark-$version-bin-hadoop3.tgz"
-case Some((3, minor, _)) if minor >= 3 => 
s"spark-$version-bin-hadoop3.tgz"
-case Some((3, minor, _)) if minor < 3 => 
s"spark-$version-bin-hadoop3.2.tgz"
-case Some((_, _, _)) => s"spark-$version-bin-hadoop2.7.tgz"
-case None => s"spark-$version-bin-hadoop2.7.tgz"
-  }
+  val filename = s"spark-$version-bin-hadoop3.tgz"
   val url = s"$site/spark/spark-$version/$filename"
   logInfo(s"Downloading Spark $version from $url")
   try {


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



(spark) branch master updated: [SPARK-45502][BUILD] Upgrade Kafka to 3.6.1

2023-12-16 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 b702f63bed27 [SPARK-45502][BUILD] Upgrade Kafka to 3.6.1
b702f63bed27 is described below

commit b702f63bed27b73bae748e232236da2f2ed19dfb
Author: dengziming 
AuthorDate: Sat Dec 16 14:17:39 2023 -0800

[SPARK-45502][BUILD] Upgrade Kafka to 3.6.1

### What changes were proposed in this pull request?
Upgrade Apache Kafka from 3.4.1 to 3.6.1

### Why are the changes needed?

- https://downloads.apache.org/kafka/3.6.1/RELEASE_NOTES.html
- https://downloads.apache.org/kafka/3.6.0/RELEASE_NOTES.html
- https://downloads.apache.org/kafka/3.5.0/RELEASE_NOTES.html

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

### How was this patch tested?
GitHub CI.

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

Closes #43348 from dengziming/kafka-3.6.0.

Authored-by: dengziming 
Signed-off-by: Dongjoon Hyun 

Closes #44312 from dengziming/kafka-3.6.1.

Authored-by: dengziming 
Signed-off-by: Dongjoon Hyun 
---
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  |  6 --
 .../apache/spark/sql/kafka010/KafkaTestUtils.scala |  4 ++--
 .../spark/streaming/kafka010/KafkaRDDSuite.scala   | 16 --
 .../spark/streaming/kafka010/KafkaTestUtils.scala  |  4 ++--
 .../streaming/kafka010/mocks/MockScheduler.scala   | 25 +++---
 pom.xml|  2 +-
 6 files changed, 30 insertions(+), 27 deletions(-)

diff --git 
a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
 
b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
index 02e4e909734a..5b4567aa2881 100644
--- 
a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
+++ 
b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala
@@ -154,7 +154,7 @@ abstract class KafkaSourceTest extends StreamTest with 
SharedSparkSession with K
   }
 
   val offset = KafkaSourceOffset(testUtils.getLatestOffsets(topics))
-  logInfo(s"Added data, expected offset $offset")
+  logInfo(s"Added data to topic: $topic, expected offset: $offset")
   (kafkaSource, offset)
 }
 
@@ -2691,7 +2691,9 @@ class KafkaSourceStressSuite extends KafkaSourceTest {
 start + Random.nextInt(start + end - 1)
   }
 
-  test("stress test with multiple topics and partitions")  {
+  override val brokerProps = Map("auto.create.topics.enable" -> "false")
+
+  test("stress test with multiple topics and partitions") {
 topics.foreach { topic =>
   testUtils.createTopic(topic, partitions = nextInt(1, 6))
   testUtils.sendMessages(topic, (101 to 105).map { _.toString }.toArray)
diff --git 
a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
 
b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
index 1fa1dda9faf2..64e54ad63bdc 100644
--- 
a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
+++ 
b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala
@@ -28,7 +28,6 @@ import scala.io.Source
 import scala.jdk.CollectionConverters._
 
 import com.google.common.io.Files
-import kafka.api.Request
 import kafka.server.{HostedPartition, KafkaConfig, KafkaServer}
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.zk.KafkaZkClient
@@ -40,6 +39,7 @@ import org.apache.kafka.clients.producer._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.config.SaslConfigs
 import org.apache.kafka.common.network.ListenerName
+import org.apache.kafka.common.requests.FetchRequest
 import org.apache.kafka.common.security.auth.SecurityProtocol.{PLAINTEXT, 
SASL_PLAINTEXT}
 import org.apache.kafka.common.serialization.StringSerializer
 import org.apache.kafka.common.utils.SystemTime
@@ -603,7 +603,7 @@ class KafkaTestUtils(
 .getPartitionInfo(topic, partition) match {
   case Some(partitionState) =>
 zkClient.getLeaderForPartition(new TopicPartition(topic, 
partition)).isDefined &&
-  Request.isValidBrokerId(partitionState.leader) &&
+  FetchRequest.isValidBrokerId(partitionState.leader) &&
   !partitionState.replicas.isEmpty
 
   case _ =>
diff --git 
a/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala
 
b/connector/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/

(spark) branch master updated: [MINOR][DOCS] Use user-facing feature names in Python metrics documentation

2023-12-16 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 20be0b0bffcb [MINOR][DOCS] Use user-facing feature names in Python 
metrics documentation
20be0b0bffcb is described below

commit 20be0b0bffcb3875c50af955ca89b9bd7d59fc8c
Author: Hyukjin Kwon 
AuthorDate: Sat Dec 16 14:14:05 2023 -0800

[MINOR][DOCS] Use user-facing feature names in Python metrics documentation

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

This PR proposes to use proper user-facing feature names instead of the 
names of logical plans in Python metrics documentation.

### Why are the changes needed?

- For better maintenance of the codes (e.g., renaming the plan)
- Users won't likely know much about the plan.

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

Yes, it changes the user-facing documentation for Python metrics 
documentation.

### How was this patch tested?

CI in this PR should verify it.

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

No.

Closes #44380 from HyukjinKwon/minor-doc-update.

Authored-by: Hyukjin Kwon 
Signed-off-by: Dongjoon Hyun 
---
 docs/web-ui.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/web-ui.md b/docs/web-ui.md
index cdf62e0d8ec0..c500860a201b 100644
--- a/docs/web-ui.md
+++ b/docs/web-ui.md
@@ -406,8 +406,8 @@ Here is the list of SQL metrics:
  time to build hash map  the time spent on 
building hash map  ShuffledHashJoin 
  task commit time  the time spent on committing 
the output of a task after the writes succeed  any write operation on 
a file-based table 
  job commit time  the time spent on committing 
the output of a job after the writes succeed  any write operation on a 
file-based table 
- data sent to Python workers  the number of 
bytes of serialized data sent to the Python workers  ArrowEvalPython, 
AggregateInPandas, BatchEvalPython, FlatMapGroupsInPandas, 
FlatMapsCoGroupsInPandas, FlatMapsCoGroupsInPandasWithState, MapInPandas, 
PythonMapInArrow, WindowsInPandas 
- data returned from Python workers  the number 
of bytes of serialized data received back from the Python workers  
ArrowEvalPython, AggregateInPandas, BatchEvalPython, FlatMapGroupsInPandas, 
FlatMapsCoGroupsInPandas, FlatMapsCoGroupsInPandasWithState, MapInPandas, 
PythonMapInArrow, WindowsInPandas 
+ data sent to Python workers  the number of 
bytes of serialized data sent to the Python workers  Python UDFs, 
Pandas UDFs, Pandas Functions API and Python Data Source 
+ data returned from Python workers  the number 
of bytes of serialized data received back from the Python workers  
Python UDFs, Pandas UDFS, Pandas Functions API and Python Data Source 
 
 
 ## Structured Streaming Tab


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



(spark) tag v3.3.4 created (now 18db204995b)

2023-12-16 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a change to tag v3.3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


  at 18db204995b (commit)
No new revisions were added by this update.


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



svn commit: r66087 - /release/spark/spark-3.3.3/

2023-12-15 Thread dongjoon
Author: dongjoon
Date: Sat Dec 16 00:33:47 2023
New Revision: 66087

Log:
Remove Apache Spark 3.3.3 after releasing 3.3.4

Removed:
release/spark/spark-3.3.3/


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



svn commit: r66086 - /dev/spark/v3.3.4-rc1-bin/ /release/spark/spark-3.3.4/

2023-12-15 Thread dongjoon
Author: dongjoon
Date: Sat Dec 16 00:25:31 2023
New Revision: 66086

Log:
Release Apache Spark 3.3.4

Added:
release/spark/spark-3.3.4/
  - copied from r66085, dev/spark/v3.3.4-rc1-bin/
Removed:
dev/spark/v3.3.4-rc1-bin/


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



(spark) branch master updated: [SPARK-46353][CORE] Refactor to improve `RegisterWorker` unit test coverage

2023-12-12 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 2215cef40043 [SPARK-46353][CORE] Refactor to improve `RegisterWorker` 
unit test coverage
2215cef40043 is described below

commit 2215cef40043a3205446f8daecafed8f2360a742
Author: Dongjoon Hyun 
AuthorDate: Tue Dec 12 09:57:43 2023 -0800

[SPARK-46353][CORE] Refactor to improve `RegisterWorker` unit test coverage

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

This PR aims to improve the unit test coverage for `RegisterWorker` message 
handling.

- Add `handleRegisterWorker` helper method which is testable easily.
- Add new unit tests for three conditional branches.

### Why are the changes needed?

It's easily to test and improve. We can add more tests in this way in the 
future.

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

No. This is a refactoring on the main code and only additions to the test 
methods.

### How was this patch tested?

Pass the CIs.

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

No.

Closes #44284 from dongjoon-hyun/SPARK-46353.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../org/apache/spark/deploy/master/Master.scala| 75 +-
 .../apache/spark/deploy/master/MasterSuite.scala   | 59 -
 2 files changed, 102 insertions(+), 32 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index a550f44fc0a4..c8679c185ad7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -37,7 +37,7 @@ import org.apache.spark.internal.config.Deploy._
 import org.apache.spark.internal.config.UI._
 import org.apache.spark.internal.config.Worker._
 import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances}
-import org.apache.spark.resource.{ResourceProfile, ResourceRequirement, 
ResourceUtils}
+import org.apache.spark.resource.{ResourceInformation, ResourceProfile, 
ResourceRequirement, ResourceUtils}
 import org.apache.spark.rpc._
 import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, Serializer}
 import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, 
Utils}
@@ -75,7 +75,8 @@ private[deploy] class Master(
   private val waitingApps = new ArrayBuffer[ApplicationInfo]
   val apps = new HashSet[ApplicationInfo]
 
-  private val idToWorker = new HashMap[String, WorkerInfo]
+  // Visible for testing
+  private[master] val idToWorker = new HashMap[String, WorkerInfo]
   private val addressToWorker = new HashMap[RpcAddress, WorkerInfo]
 
   private val endpointToApp = new HashMap[RpcEndpointRef, ApplicationInfo]
@@ -106,7 +107,7 @@ private[deploy] class Master(
 
   private[master] var state = RecoveryState.STANDBY
 
-  private var persistenceEngine: PersistenceEngine = _
+  private[master] var persistenceEngine: PersistenceEngine = _
 
   private var leaderElectionAgent: LeaderElectionAgent = _
 
@@ -281,33 +282,8 @@ private[deploy] class Master(
 case RegisterWorker(
   id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl,
   masterAddress, resources) =>
-  logInfo("Registering worker %s:%d with %d cores, %s RAM".format(
-workerHost, workerPort, cores, Utils.megabytesToString(memory)))
-  if (state == RecoveryState.STANDBY) {
-workerRef.send(MasterInStandby)
-  } else if (idToWorker.contains(id)) {
-if (idToWorker(id).state == WorkerState.UNKNOWN) {
-  logInfo("Worker has been re-registered: " + id)
-  idToWorker(id).state = WorkerState.ALIVE
-}
-workerRef.send(RegisteredWorker(self, masterWebUiUrl, masterAddress, 
true))
-  } else {
-val workerResources =
-  resources.map(r => r._1 -> WorkerResourceInfo(r._1, 
r._2.addresses.toImmutableArraySeq))
-val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory,
-  workerRef, workerWebUiUrl, workerResources)
-if (registerWorker(worker)) {
-  persistenceEngine.addWorker(worker)
-  workerRef.send(RegisteredWorker(self, masterWebUiUrl, masterAddress, 
false))
-  schedule()
-} else {
-  val workerAddress = worker.endpoint.address
-  logWarning("Worker registration failed. Attempted to re-register 
worker at same " +
-"address: " + workerAddress)
-  workerRef.send(RegisterWorkerFailed("Attempted to re-register worker 
at same address: "
-+ workerAddress))
-}
-  }
+  handle

(spark) branch branch-3.4 updated: [SPARK-46369][CORE] Remove `kill` link from `RELAUNCHING` drivers in `MasterPage`

2023-12-11 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new b813e2e100f [SPARK-46369][CORE] Remove `kill` link from `RELAUNCHING` 
drivers in `MasterPage`
b813e2e100f is described below

commit b813e2e100faf7bab88c23ba9bba6e3197b169aa
Author: Dongjoon Hyun 
AuthorDate: Mon Dec 11 15:05:21 2023 -0800

[SPARK-46369][CORE] Remove `kill` link from `RELAUNCHING` drivers in 
`MasterPage`

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

This PR aims to remove `kill` hyperlink from `RELAUNCHING` drivers in 
`MasterPage`.

### Why are the changes needed?

Since Apache Spark 1.4.0 (SPARK-5495), `RELAUNCHING` drivers have `kill` 
hyperlinks in the `Completed Drivers` table.

![Screenshot 2023-12-11 at 1 02 29 
PM](https://github.com/apache/spark/assets/9700541/38f4bf08-efb9-47e5-8a7a-f7d127429012)

However, this is a bug because the driver was already terminated by 
definition. Newly relaunched driver has an independent ID and there is no 
relationship with the previously terminated ID.


https://github.com/apache/spark/blob/7db85642600b1e3b39ca11e41d4e3e0bf1c8962b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala#L27

If we clicked the `kill` link, `Master` always complains like the following.
```
23/12/11 21:25:50 INFO Master: Asked to kill driver 202312112113-0
23/12/11 21:25:50 WARN Master: Driver 202312112113-0 has already 
finished or does not exist
```

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

No.

### How was this patch tested?

Manual review.

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

No.

Closes #44301 from dongjoon-hyun/SPARK-46369.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit e434c9f0d5792b7af43c87dd6145fd8a6a04d8e2)
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit ac031d68a01f14cc73f05e83a790a6787aa6453d)
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index a71eb33a2fe..e7e90aa0a37 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -322,8 +322,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends 
WebUIPage("") {
   private def driverRow(driver: DriverInfo, showDuration: Boolean): Seq[Node] 
= {
 val killLink = if (parent.killEnabled &&
   (driver.state == DriverState.RUNNING ||
-driver.state == DriverState.SUBMITTED ||
-driver.state == DriverState.RELAUNCHING)) {
+driver.state == DriverState.SUBMITTED)) {
   val confirm =
 s"if (window.confirm('Are you sure you want to kill driver 
${driver.id} ?')) " +
   "{ this.parentNode.submit(); return true; } else { return false; }"


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



(spark) branch branch-3.5 updated: [SPARK-46369][CORE] Remove `kill` link from `RELAUNCHING` drivers in `MasterPage`

2023-12-11 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new ac031d68a01 [SPARK-46369][CORE] Remove `kill` link from `RELAUNCHING` 
drivers in `MasterPage`
ac031d68a01 is described below

commit ac031d68a01f14cc73f05e83a790a6787aa6453d
Author: Dongjoon Hyun 
AuthorDate: Mon Dec 11 15:05:21 2023 -0800

[SPARK-46369][CORE] Remove `kill` link from `RELAUNCHING` drivers in 
`MasterPage`

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

This PR aims to remove `kill` hyperlink from `RELAUNCHING` drivers in 
`MasterPage`.

### Why are the changes needed?

Since Apache Spark 1.4.0 (SPARK-5495), `RELAUNCHING` drivers have `kill` 
hyperlinks in the `Completed Drivers` table.

![Screenshot 2023-12-11 at 1 02 29 
PM](https://github.com/apache/spark/assets/9700541/38f4bf08-efb9-47e5-8a7a-f7d127429012)

However, this is a bug because the driver was already terminated by 
definition. Newly relaunched driver has an independent ID and there is no 
relationship with the previously terminated ID.


https://github.com/apache/spark/blob/7db85642600b1e3b39ca11e41d4e3e0bf1c8962b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala#L27

If we clicked the `kill` link, `Master` always complains like the following.
```
23/12/11 21:25:50 INFO Master: Asked to kill driver 202312112113-0
23/12/11 21:25:50 WARN Master: Driver 202312112113-0 has already 
finished or does not exist
```

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

No.

### How was this patch tested?

Manual review.

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

No.

Closes #44301 from dongjoon-hyun/SPARK-46369.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit e434c9f0d5792b7af43c87dd6145fd8a6a04d8e2)
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index a71eb33a2fe..e7e90aa0a37 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -322,8 +322,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends 
WebUIPage("") {
   private def driverRow(driver: DriverInfo, showDuration: Boolean): Seq[Node] 
= {
 val killLink = if (parent.killEnabled &&
   (driver.state == DriverState.RUNNING ||
-driver.state == DriverState.SUBMITTED ||
-driver.state == DriverState.RELAUNCHING)) {
+driver.state == DriverState.SUBMITTED)) {
   val confirm =
 s"if (window.confirm('Are you sure you want to kill driver 
${driver.id} ?')) " +
   "{ this.parentNode.submit(); return true; } else { return false; }"


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



(spark) branch master updated (3e0808c33f1 -> e434c9f0d57)

2023-12-11 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 3e0808c33f1 [SPARK-46351][SQL] Require an error class in 
`AnalysisException`
 add e434c9f0d57 [SPARK-46369][CORE] Remove `kill` link from `RELAUNCHING` 
drivers in `MasterPage`

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala | 3 +--
 1 file changed, 1 insertion(+), 2 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-46358][CONNECT] Simplify the condition check in the `ResponseValidator#verifyResponse`

2023-12-11 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 051b1781827 [SPARK-46358][CONNECT] Simplify the condition check in the 
`ResponseValidator#verifyResponse`
051b1781827 is described below

commit 051b1781827dd3a4e1e95a5354caa747ff41ae1a
Author: yangjie01 
AuthorDate: Mon Dec 11 08:53:10 2023 -0800

[SPARK-46358][CONNECT] Simplify the condition check in the 
`ResponseValidator#verifyResponse`

### What changes were proposed in this pull request?
This PR has made the following refactoring to the `verifyResponse` function 
in `ResponseValidator`:
1. The check condition `response.hasField(field)` is moved before getting 
`value`, and only when `response.hasField(field)` is true, `value` is obtained, 
which seems more in line with the existing comments.
2. Removed the `value != ""` condition check in the case match, because 
only when `value.nonEmpty` is true will it enter the `if` branch, and the 
condition `value.nonEmpty` has already covered the check for `value != ""`.
3. The condition check `value != id` is moved inside `case Some(id)`. After 
the modification, an `IllegalStateException` will still be thrown when the id 
exists and `value != id`, but `serverSideSessionId` will no longer be 
reassigned when the id exists and `value == id`.
4. Removed the redundant `toString` operation when reassigning 
`serverSideSessionId`, because `value` is String type.
5. Removed the No-op `case _` match, because it is unreachable code after 
the above modifications.

### Why are the changes needed?
Simplify the condition check in the `verifyResponse` function

### 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 #44291 from LuciferYang/Simplify-ResponseValidator-verifyResponse.

Lead-authored-by: yangjie01 
Co-authored-by: YangJie 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/sql/connect/client/ResponseValidator.scala | 16 +---
 1 file changed, 9 insertions(+), 7 deletions(-)

diff --git 
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala
 
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala
index 67f29c727ef..22c5505e7d4 100644
--- 
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala
+++ 
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala
@@ -35,18 +35,20 @@ class ResponseValidator extends Logging {
 val field = 
response.getDescriptorForType.findFieldByName("server_side_session_id")
 // If the field does not exist, we ignore it. New / Old message might not 
contain it and this
 // behavior allows us to be compatible.
-if (field != null) {
+if (field != null && response.hasField(field)) {
   val value = response.getField(field).asInstanceOf[String]
   // Ignore, if the value is unset.
-  if (response.hasField(field) && value != null && value.nonEmpty) {
+  if (value != null && value.nonEmpty) {
 serverSideSessionId match {
-  case Some(id) if value != id && value != "" =>
-throw new IllegalStateException(s"Server side session ID changed 
from $id to $value")
-  case _ if value != "" =>
+  case Some(id) =>
+if (value != id) {
+  throw new IllegalStateException(
+s"Server side session ID changed from $id to $value")
+}
+  case _ =>
 synchronized {
-  serverSideSessionId = Some(value.toString)
+  serverSideSessionId = Some(value)
 }
-  case _ => // No-op
 }
   }
 } else {


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



(spark) branch master updated (bab884082c0 -> bb886abcc4a)

2023-12-11 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 bab884082c0 [MINOR][DOCS] Fix documentation for 
`spark.sql.legacy.doLooseUpcast` in SQL migration guide
 add bb886abcc4a [SPARK-46356][BUILD] Upgrade `sbt-assembly` to 2.1.5, 
`sbt-checkstyle-plugin` to 4.0.1

No new revisions were added by this update.

Summary of changes:
 project/plugins.sbt | 4 ++--
 1 file changed, 2 insertions(+), 2 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-46215][CORE][FOLLOWUP] Handle symbolic links

2023-12-10 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 bacdb3b5fec [SPARK-46215][CORE][FOLLOWUP] Handle symbolic links
bacdb3b5fec is described below

commit bacdb3b5fec9783f4604276480eb2a0f5702
Author: Dongjoon Hyun 
AuthorDate: Sun Dec 10 18:46:37 2023 -0800

[SPARK-46215][CORE][FOLLOWUP] Handle symbolic links

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

This PR aims to fix a regression on symbolic links.

### Why are the changes needed?

To have the same behavior with symbolic links.

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

No.

### How was this patch tested?

Pass the CIs with newly added test case.

I also verified this on Mac's `/tmp` directory.
```
$ ls -al /tmp
lrwxr-xr-x 1 root  wheel  11 Nov 17 02:37 /tmp -> private/tmp
```

**MASTER**
```
23/12/10 16:04:53 INFO FileSystemRecoveryModeFactory: Persisting recovery 
state to directory: /tmp
23/12/10 16:04:53 INFO Master: I have been elected leader! New state: ALIVE
23/12/10 16:08:39 INFO Master: Registering worker 127.0.0.1:50535 with 8 
cores, 15.0 GiB RAM
```

**PERSISTED DATA**
```
$ ls -al /tmp/worker_*
-rw-r--r-- 1 dongjoon  wheel  1354 Dec 10 16:08 
/tmp/worker_worker-20231210160839-127.0.0.1-50535
```

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

No.

Closes #44285 from dongjoon-hyun/SPARK-46215-2.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../deploy/master/FileSystemPersistenceEngine.scala  |  9 +++--
 .../spark/deploy/master/PersistenceEngineSuite.scala | 16 
 2 files changed, 23 insertions(+), 2 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
 
b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
index 785367a0dee..fb067f10c5a 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.deploy.master
 
 import java.io._
-import java.nio.file.{Files, Paths}
+import java.nio.file.{FileAlreadyExistsException, Files, Paths}
 
 import scala.reflect.ClassTag
 
@@ -42,7 +42,12 @@ private[master] class FileSystemPersistenceEngine(
 val codec: Option[CompressionCodec] = None)
   extends PersistenceEngine with Logging {
 
-  Files.createDirectories(Paths.get(dir))
+  try {
+Files.createDirectories(Paths.get(dir))
+  } catch {
+case _: FileAlreadyExistsException if Files.isSymbolicLink(Paths.get(dir)) 
=>
+  Files.createDirectories(Paths.get(dir).toRealPath())
+  }
 
   override def persist(name: String, obj: Object): Unit = {
 serializeIntoFile(new File(dir + File.separator + name), obj)
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala
index 84181ea3fca..b977a114244 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala
@@ -19,6 +19,7 @@
 package org.apache.spark.deploy.master
 
 import java.net.ServerSocket
+import java.nio.file.{Files, Paths}
 import java.util.concurrent.ThreadLocalRandom
 
 import org.apache.curator.test.TestingServer
@@ -72,6 +73,21 @@ class PersistenceEngineSuite extends SparkFunSuite {
 }
   }
 
+  test("SPARK-46215: FileSystemPersistenceEngine with a symbolic link") {
+withTempDir { dir =>
+  val target = Paths.get(dir.getAbsolutePath(), "target")
+  val link = Paths.get(dir.getAbsolutePath(), "symbolic_link");
+
+  Files.createDirectories(target)
+  Files.createSymbolicLink(link, target);
+
+  val conf = new SparkConf()
+  testPersistenceEngine(conf, serializer =>
+new FileSystemPersistenceEngine(link.toAbsolutePath.toString, 
serializer)
+  )
+}
+  }
+
   test("SPARK-46205: Support KryoSerializer in FileSystemPersistenceEngine") {
 withTempDir { dir =>
   val conf = new SparkConf()


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



(spark) branch master updated: [SPARK-45642][CORE][SQL] Fix `FileSystem.isFile & FileSystem.isDirectory is deprecated`

2023-12-10 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 fd009d652f7 [SPARK-45642][CORE][SQL] Fix `FileSystem.isFile & 
FileSystem.isDirectory is deprecated`
fd009d652f7 is described below

commit fd009d652f7922254ccc7cc631b8df3a6b821532
Author: panbingkun 
AuthorDate: Sun Dec 10 14:11:19 2023 -0800

[SPARK-45642][CORE][SQL] Fix `FileSystem.isFile & FileSystem.isDirectory is 
deprecated`

### What changes were proposed in this pull request?
The pr aims to fix `FileSystem.isFile & FileSystem.isDirectory is 
deprecated` & make some error message prompts more accurate.

### Why are the changes needed?
- Prepare for future Hadoop to truly eliminate this method
- Reduce warn prompts.
- Make some error message prompts more accurate.

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

### How was this patch tested?
- Pass GA.
- Manually test

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

Closes #43505 from panbingkun/SPARK-45642.

Authored-by: panbingkun 
    Signed-off-by: Dongjoon Hyun 
---
 .../scala/org/apache/spark/deploy/SparkHadoopUtil.scala|  9 -
 .../apache/spark/deploy/history/FsHistoryProvider.scala|  2 +-
 core/src/main/scala/org/apache/spark/util/Utils.scala  |  2 +-
 .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala   |  2 +-
 .../execution/datasources/PartitioningAwareFileIndex.scala | 14 +++---
 .../spark/sql/execution/streaming/FileStreamSink.scala |  2 +-
 .../scala/org/apache/spark/streaming/util/HdfsUtils.scala  |  2 +-
 7 files changed, 24 insertions(+), 9 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala 
b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 50906f76b6e..628b688dedb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.deploy
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, 
DataOutputStream, File, IOException}
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, 
DataOutputStream, File, FileNotFoundException, IOException}
 import java.net.InetAddress
 import java.security.PrivilegedExceptionAction
 import java.text.DateFormat
@@ -593,4 +593,11 @@ private[spark] object SparkHadoopUtil extends Logging {
 }
   }
 
+  def isFile(fs: FileSystem, path: Path): Boolean = {
+try {
+  fs.getFileStatus(path).isFile
+} catch {
+  case _: FileNotFoundException => false
+}
+  }
 }
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala 
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index 565499bb610..73fb0086b33 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -860,7 +860,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
   try {
 // Fetch the entry first to avoid an RPC when it's already removed.
 listing.read(classOf[LogInfo], inProgressLog)
-if (!fs.isFile(new Path(inProgressLog))) {
+if (!SparkHadoopUtil.isFile(fs, new Path(inProgressLog))) {
   listing.synchronized {
 listing.delete(classOf[LogInfo], inProgressLog)
   }
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala 
b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 04281f5..a074bd53d26 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -674,7 +674,7 @@ private[spark] object Utils
   throw new IOException(s"Failed to create directory ${targetDir.getPath}")
 }
 val dest = new File(targetDir, filename.getOrElse(path.getName))
-if (fs.isFile(path)) {
+if (fs.getFileStatus(path).isFile) {
   val in = fs.open(path)
   try {
 downloadFile(path.toString, in, dest, fileOverwrite)
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala 
b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index e235b8aeb77..d16a15df1b5 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -1774,7 +1774,7 @@ class TestFileSystem extends 
org.apache.hadoop.fs.LocalFileSystem {
 status
   }
 
-  override def isFile(path: Path): Boolean = super.isFile(local(path))
+  override def getFileS

(spark) branch branch-3.5 updated: [SPARK-45969][DOCS] Document configuration change of executor failure tracker

2023-12-10 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new cbaefe9cc6a [SPARK-45969][DOCS] Document configuration change of 
executor failure tracker
cbaefe9cc6a is described below

commit cbaefe9cc6a22c940728b6717aeaa51c7d550ddc
Author: Cheng Pan 
AuthorDate: Sun Dec 10 14:03:37 2023 -0800

[SPARK-45969][DOCS] Document configuration change of executor failure 
tracker

It's a follow-up of SPARK-41210 (use a new JIRA ticket because it was 
released in 3.5.0), this PR updates docs/migration guide about configuration 
change of executor failure tracker

Docs update is missing in previous changes, also is requested 
https://github.com/apache/spark/commit/40872e9a094f8459b0b6f626937ced48a8d98efb#r132516892
 by tgravescs

Yes, docs changed

Review

No

Closes #43863 from pan3793/SPARK-45969.

Authored-by: Cheng Pan 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 7a43de193aa5a0856e098088728dccea37f169c5)
Signed-off-by: Dongjoon Hyun 
---
 .../org/apache/spark/internal/config/package.scala  |  4 ++--
 docs/configuration.md   | 21 +
 docs/core-migration-guide.md|  6 ++
 docs/running-on-yarn.md | 17 -
 4 files changed, 29 insertions(+), 19 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala 
b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 600cbf151e1..c5e23cae1f8 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -924,7 +924,7 @@ package object config {
 
   private[spark] val MAX_EXECUTOR_FAILURES =
 ConfigBuilder("spark.executor.maxNumFailures")
-  .doc("Spark exits if the number of failed executors exceeds this 
threshold. " +
+  .doc("The maximum number of executor failures before failing the 
application. " +
 "This configuration only takes effect on YARN, or Kubernetes when " +
 "`spark.kubernetes.allocation.pods.allocator` is set to 'direct'.")
   .version("3.5.0")
@@ -933,7 +933,7 @@ package object config {
 
   private[spark] val EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS =
 ConfigBuilder("spark.executor.failuresValidityInterval")
-  .doc("Interval after which Executor failures will be considered 
independent and not " +
+  .doc("Interval after which executor failures will be considered 
independent and not " +
 "accumulate towards the attempt count. This configuration only takes 
effect on YARN, " +
 "or Kubernetes when `spark.kubernetes.allocation.pods.allocator` is 
set to 'direct'.")
   .version("3.5.0")
diff --git a/docs/configuration.md b/docs/configuration.md
index f79406c5b6d..645c3e8208a 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -514,6 +514,27 @@ of the most common options to set are:
   
   3.2.0
 
+
+  spark.executor.maxNumFailures
+  numExecutors * 2, with minimum of 3
+  
+The maximum number of executor failures before failing the application.
+This configuration only takes effect on YARN, or Kubernetes when 
+`spark.kubernetes.allocation.pods.allocator` is set to 'direct'.
+  
+  3.5.0
+
+
+  spark.executor.failuresValidityInterval
+  (none)
+  
+Interval after which executor failures will be considered independent and
+not accumulate towards the attempt count.
+This configuration only takes effect on YARN, or Kubernetes when 
+`spark.kubernetes.allocation.pods.allocator` is set to 'direct'.
+  
+  3.5.0
+
 
 
 Apart from these, the following properties are also available, and may be 
useful in some situations:
diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md
index 3f97a484e1a..36465cc3f4e 100644
--- a/docs/core-migration-guide.md
+++ b/docs/core-migration-guide.md
@@ -22,6 +22,12 @@ license: |
 * Table of contents
 {:toc}
 
+## Upgrading from Core 3.4 to 3.5
+
+- Since Spark 3.5, `spark.yarn.executor.failuresValidityInterval` is 
deprecated. Use `spark.executor.failuresValidityInterval` instead.
+
+- Since Spark 3.5, `spark.yarn.max.executor.failures` is deprecated. Use 
`spark.executor.maxNumFailures` instead.
+
 ## Upgrading from Core 3.3 to 3.4
 
 - Since Spark 3.4, Spark driver will own `PersistentVolumnClaim`s and try to 
reuse if they are not assigned to live executors. To restore the behavior 
before Spark 3.4, you can set 
`spark.kubernetes.driver.ownPersistentVolumeClaim` to `false` and 
`spark.kubernetes.driver.reusePersistentVolumeClaim` to `false`.

(spark) branch master updated: [SPARK-45969][DOCS] Document configuration change of executor failure tracker

2023-12-10 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 7a43de193aa [SPARK-45969][DOCS] Document configuration change of 
executor failure tracker
7a43de193aa is described below

commit 7a43de193aa5a0856e098088728dccea37f169c5
Author: Cheng Pan 
AuthorDate: Sun Dec 10 14:03:37 2023 -0800

[SPARK-45969][DOCS] Document configuration change of executor failure 
tracker

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

It's a follow-up of SPARK-41210 (use a new JIRA ticket because it was 
released in 3.5.0), this PR updates docs/migration guide about configuration 
change of executor failure tracker

### Why are the changes needed?

Docs update is missing in previous changes, also is requested 
https://github.com/apache/spark/commit/40872e9a094f8459b0b6f626937ced48a8d98efb#r132516892
 by tgravescs

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

Yes, docs changed

### How was this patch tested?

Review

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

No

Closes #43863 from pan3793/SPARK-45969.

Authored-by: Cheng Pan 
Signed-off-by: Dongjoon Hyun 
---
 .../org/apache/spark/internal/config/package.scala  |  4 ++--
 docs/configuration.md   | 21 +
 docs/core-migration-guide.md|  6 ++
 docs/running-on-yarn.md | 17 -
 4 files changed, 29 insertions(+), 19 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala 
b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 2c710e6025d..2823b7cdb60 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -931,7 +931,7 @@ package object config {
 
   private[spark] val MAX_EXECUTOR_FAILURES =
 ConfigBuilder("spark.executor.maxNumFailures")
-  .doc("Spark exits if the number of failed executors exceeds this 
threshold. " +
+  .doc("The maximum number of executor failures before failing the 
application. " +
 "This configuration only takes effect on YARN, or Kubernetes when " +
 "`spark.kubernetes.allocation.pods.allocator` is set to 'direct'.")
   .version("3.5.0")
@@ -940,7 +940,7 @@ package object config {
 
   private[spark] val EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS =
 ConfigBuilder("spark.executor.failuresValidityInterval")
-  .doc("Interval after which Executor failures will be considered 
independent and not " +
+  .doc("Interval after which executor failures will be considered 
independent and not " +
 "accumulate towards the attempt count. This configuration only takes 
effect on YARN, " +
 "or Kubernetes when `spark.kubernetes.allocation.pods.allocator` is 
set to 'direct'.")
   .version("3.5.0")
diff --git a/docs/configuration.md b/docs/configuration.md
index f261e3b2deb..b45d647fde8 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -522,6 +522,27 @@ of the most common options to set are:
   
   3.2.0
 
+
+  spark.executor.maxNumFailures
+  numExecutors * 2, with minimum of 3
+  
+The maximum number of executor failures before failing the application.
+This configuration only takes effect on YARN, or Kubernetes when 
+`spark.kubernetes.allocation.pods.allocator` is set to 'direct'.
+  
+  3.5.0
+
+
+  spark.executor.failuresValidityInterval
+  (none)
+  
+Interval after which executor failures will be considered independent and
+not accumulate towards the attempt count.
+This configuration only takes effect on YARN, or Kubernetes when 
+`spark.kubernetes.allocation.pods.allocator` is set to 'direct'.
+  
+  3.5.0
+
 
 
 Apart from these, the following properties are also available, and may be 
useful in some situations:
diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md
index 09ba4b474e9..179b0b3fae1 100644
--- a/docs/core-migration-guide.md
+++ b/docs/core-migration-guide.md
@@ -32,6 +32,12 @@ license: |
 
 - In Spark 4.0, support for Apache Mesos as a resource manager was removed.
 
+## Upgrading from Core 3.4 to 3.5
+
+- Since Spark 3.5, `spark.yarn.executor.failuresValidityInterval` is 
deprecated. Use `spark.executor.failuresValidityInterval` instead.
+
+- Since Spark 3.5, `spark.yarn.max.executor.failures` is deprecated. Use 
`spark.executor.maxNumFailures` instead.
+
 ## Upgrading from Core 3.3 to 3.4
 
 - Since Spark 3.4, Spark driver will own `PersistentVolumnClai

(spark) branch master updated: [SPARK-46285][SQL] Add `foreachWithSubqueries`

2023-12-10 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 a77713081a8 [SPARK-46285][SQL] Add `foreachWithSubqueries`
a77713081a8 is described below

commit a77713081a8065a5feaa3b438a79fdeb6a4b6782
Author: Rui Wang 
AuthorDate: Sun Dec 10 14:00:15 2023 -0800

[SPARK-46285][SQL] Add `foreachWithSubqueries`

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

We can have a `foreachWithSubqueries` which also traverse the subqueries in 
the query plan.

### Why are the changes needed?

Add a new way to access subqueries in the query plan.

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

no

### How was this patch tested?

UT

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

no

Closes #44206 from amaliujia/foreachsubqueries.

Authored-by: Rui Wang 
Signed-off-by: Dongjoon Hyun 
---
 .../org/apache/spark/sql/catalyst/plans/QueryPlan.scala | 11 +++
 .../apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala  | 13 +
 2 files changed, 24 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index 9656a93575d..ef7cd7401f2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -534,6 +534,17 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]]
 transformDownWithPruning(cond, ruleId)(g)
   }
 
+  /**
+   * A variant of [[foreach]] which considers plan nodes inside subqueries as 
well.
+   */
+  def foreachWithSubqueries(f: PlanType => Unit): Unit = {
+def actualFunc(plan: PlanType): Unit = {
+  f(plan)
+  plan.subqueries.foreach(_.foreachWithSubqueries(f))
+}
+foreach(actualFunc)
+  }
+
   /**
* A variant of `collect`. This method not only apply the given function to 
all elements in this
* plan, also considering all the plans in its (nested) subqueries
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
index 3eba9eebc3d..31f7e07143c 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.catalyst.plans
 
 import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
 import org.apache.spark.sql.catalyst.expressions._
@@ -145,4 +146,16 @@ class LogicalPlanSuite extends SparkFunSuite {
 assert(query.where(Literal.FalseLiteral).maxRows.contains(0))
 assert(query.where(Literal.FalseLiteral).maxRowsPerPartition.contains(0))
   }
+
+  test("SPARK-46285: foreachWithSubqueries") {
+val input = UnresolvedRelation(Seq("subquery_table"))
+val input2 = UnresolvedRelation(Seq("t"))
+val plan = Filter(Exists(input), input2)
+val tableNames = scala.collection.mutable.Set[String]()
+plan.foreachWithSubqueries {
+  case e: UnresolvedRelation => tableNames.add(e.name)
+  case _ =>
+}
+assert(tableNames.contains("subquery_table"))
+  }
 }


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



(spark) branch master updated: [SPARK-46342][SQL] Replace `IllegalStateException` by `SparkException.internalError` in sql

2023-12-10 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 d02fbba6491 [SPARK-46342][SQL] Replace `IllegalStateException` by 
`SparkException.internalError` in sql
d02fbba6491 is described below

commit d02fbba6491fd17dc6bfc1a416971af7544952f3
Author: Max Gekk 
AuthorDate: Sun Dec 10 11:24:02 2023 -0800

[SPARK-46342][SQL] Replace `IllegalStateException` by 
`SparkException.internalError` in sql

### What changes were proposed in this pull request?
In the PR, I propose to replace all `IllegalStateException` exception in 
the `sql` project except of `streaming` by `SparkException.internalError`.

### Why are the changes needed?
This is a part of migration onto new error framework and error classes.

### Does this PR introduce _any_ user-facing change?
No, users shouldn't face to `IllegalStateException` in regular cases.

### How was this patch tested?
Using existing GAs.

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

Closes #44275 from MaxGekk/replace-ise-by-internal-error.

Authored-by: Max Gekk 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/sql/catalyst/util/TimestampFormatter.scala|  5 +++--
 .../scala/org/apache/spark/sql/util/ArrowUtils.scala|  3 ++-
 .../execution/datasources/v2/DataSourceV2Relation.scala |  2 +-
 .../org/apache/spark/sql/util/ArrowUtilsSuite.scala | 12 +++-
 .../main/scala/org/apache/spark/sql/SparkSession.scala  |  6 +++---
 .../apache/spark/sql/api/python/PythonSQLUtils.scala|  3 ++-
 .../sql/catalyst/analysis/ResolveSessionCatalog.scala   |  3 ++-
 .../spark/sql/execution/OptimizeMetadataOnlyQuery.scala |  3 ++-
 .../org/apache/spark/sql/execution/SQLExecution.scala   |  4 ++--
 .../org/apache/spark/sql/execution/SparkSqlParser.scala |  3 ++-
 .../apache/spark/sql/execution/SparkStrategies.scala| 17 +
 .../spark/sql/execution/WholeStageCodegenExec.scala |  4 ++--
 .../sql/execution/adaptive/AQEShuffleReadExec.scala |  8 
 .../spark/sql/execution/adaptive/QueryStageExec.scala   |  8 
 .../execution/aggregate/AggregateCodegenSupport.scala   |  3 ++-
 .../sql/execution/aggregate/BaseAggregateExec.scala |  7 ---
 .../execution/aggregate/ObjectAggregationIterator.scala |  4 ++--
 .../aggregate/TungstenAggregationIterator.scala |  4 ++--
 .../sql/execution/aggregate/UpdatingSessionsExec.scala  |  3 ++-
 .../execution/analysis/DetectAmbiguousSelfJoin.scala|  4 +++-
 .../spark/sql/execution/basicPhysicalOperators.scala|  8 
 .../spark/sql/execution/columnar/InMemoryRelation.scala |  4 ++--
 .../columnar/compression/compressionSchemes.scala   |  5 +++--
 .../spark/sql/execution/datasources/DataSource.scala|  4 ++--
 .../sql/execution/datasources/DataSourceUtils.scala |  6 +++---
 .../sql/execution/datasources/jdbc/DriverRegistry.scala |  3 ++-
 .../datasources/parquet/ParquetWriteSupport.scala   |  4 ++--
 .../execution/datasources/v2/DataSourceV2Strategy.scala |  6 --
 .../sql/execution/exchange/ShuffleExchangeExec.scala|  4 ++--
 .../apache/spark/sql/execution/metric/SQLMetrics.scala  |  4 ++--
 .../spark/sql/execution/python/EvaluatePython.scala |  5 +++--
 .../spark/sql/execution/python/ExtractPythonUDFs.scala  |  7 ---
 .../python/FlatMapGroupsInPandasWithStateExec.scala |  4 ++--
 .../spark/sql/execution/window/AggregateProcessor.scala |  3 ++-
 .../execution/window/WindowEvaluatorFactoryBase.scala   |  9 +
 .../apache/spark/sql/expressions/ReduceAggregator.scala |  3 ++-
 .../org/apache/spark/sql/SparkSessionBuilderSuite.scala |  2 +-
 .../org/apache/spark/sql/execution/SparkPlanSuite.scala | 13 -
 .../spark/sql/execution/WholeStageCodegenSuite.scala| 13 -
 .../sql/execution/adaptive/AdaptiveQueryExecSuite.scala | 12 +++-
 .../spark/sql/expressions/ReduceAggregatorSuite.scala   |  4 ++--
 .../org/apache/spark/sql/hive/HiveInspectors.scala  |  3 ++-
 .../apache/spark/sql/hive/client/HiveClientImpl.scala   |  2 +-
 .../apache/spark/sql/hive/execution/HiveTempPath.scala  |  5 +++--
 44 files changed, 136 insertions(+), 103 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
index df146e0dbfd..9539ced52dc 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala
@@ -29,6 +29,7 @@ import scala.util.control.NonFatal
 
 import org.apache.commons.lang3.time.FastDateFormat
 
+import org.apache.spark.SparkException
 import

(spark) branch master updated: [SPARK-46346][CORE] Fix Master to update a worker from `UNKNOWN` to `ALIVE` on `RegisterWorker` msg

2023-12-10 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 109b1e4a79d [SPARK-46346][CORE] Fix Master to update a worker from 
`UNKNOWN` to `ALIVE` on `RegisterWorker` msg
109b1e4a79d is described below

commit 109b1e4a79d9a5ec33944887a34c92d453016902
Author: Dongjoon Hyun 
AuthorDate: Sun Dec 10 11:19:46 2023 -0800

[SPARK-46346][CORE] Fix Master to update a worker from `UNKNOWN` to `ALIVE` 
on `RegisterWorker` msg

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

This PR aims to fix `Spark Master`'s recovery process to update a worker 
status from `UNKNOWN` to `ALIVE` when it receives a `RegisterWroker` message 
from that worker.

### Why are the changes needed?

This only happens during the recovery.
- `Master` already has the recovered worker information in memory with 
`UNKNOWN` status.
- `Worker` sends `RegisterWorker` message correctly.
- `Master` keeps its worker status in `UNKNOWN` and informs the worker with 
`RegisteredWorker` message with `duplicated` flag.
- Since `Worker` received like the following and will not try to reconnect.
```
23/12/09 23:49:57 INFO Worker: Retrying connection to master (attempt # 3)
23/12/09 23:49:57 INFO Worker: Connecting to master ...:7077...
23/12/09 23:50:04 INFO TransportClientFactory: Successfully created 
connection to master...:7077 after 7089 ms (0 ms spent in bootstraps)
23/12/09 23:50:04 WARN Worker: Duplicate registration at master spark://...
23/12/09 23:50:04 INFO Worker: Successfully registered with master 
spark://...
```

The `UNKNOWN`-status workers blocks the recovery process and causes a long 
delay.


https://github.com/apache/spark/blob/bac3492980a3e793065a9e9d511ddf0fb66357b3/core/src/main/scala/org/apache/spark/deploy/master/Master.scala#L604-L606

After the delay, master simply kills them all.


https://github.com/apache/spark/blob/bac3492980a3e793065a9e9d511ddf0fb66357b3/core/src/main/scala/org/apache/spark/deploy/master/Master.scala#L647-L649

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

No.

### How was this patch tested?

This case is a little hard to make a unit test.
Manually test.

- Master
```
23/12/10 04:58:30 WARN OneWayOutboxMessage: Failed to send one-way RPC.
java.io.IOException: Connecting to /***:1024 timed out (1 ms)
at 
org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:291)
at 
org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:214)
at 
org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:226)
at 
org.apache.spark.rpc.netty.NettyRpcEnv.createClient(NettyRpcEnv.scala:204)
at org.apache.spark.rpc.netty.Outbox$$anon$1.call(Outbox.scala:202)
at org.apache.spark.rpc.netty.Outbox$$anon$1.call(Outbox.scala:198)
at 
java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:840)
23/12/10 04:58:54 INFO Master: Registering worker ***:1024 with 2 cores, 
23.0 GiB RAM
23/12/10 04:58:54 INFO Master: Worker has been re-registered: 
worker-20231210045613-***-1024
```

- Worker
```
23/12/10 04:58:45 INFO Worker: Retrying connection to master (attempt # 5)
23/12/10 04:58:45 INFO Worker: Connecting to master master:7077...
23/12/10 04:58:54 INFO TransportClientFactory: Successfully created 
connection to master/...:7077 after 63957 ms (0 ms spent in bootstraps)
23/12/10 04:58:54 WARN Worker: Duplicate registration at master 
spark://master:7077
23/12/10 04:58:54 INFO Worker: Successfully registered with master 
spark://master:7077
23/12/10 04:58:54 INFO Worker: WorkerWebUI is available at 
https://...-1***-1024
23/12/10 04:58:54 INFO Worker: Worker cleanup enabled; old application 
directories will be deleted in: /data/spark
```

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

No.

Closes #44280 from dongjoon-hyun/SPARK-46346.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 4 
 1 file changed, 4 insertions(+)

diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 7346c80aff4..a550f44fc0a

(spark) branch master updated: [SPARK-46348][CORE] Support `spark.deploy.recoveryTimeout`

2023-12-10 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 82a02321b87 [SPARK-46348][CORE] Support `spark.deploy.recoveryTimeout`
82a02321b87 is described below

commit 82a02321b873cc67a521e98e25b9b07fd84b5684
Author: Dongjoon Hyun 
AuthorDate: Sun Dec 10 00:44:33 2023 -0800

[SPARK-46348][CORE] Support `spark.deploy.recoveryTimeout`

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

This PR aims to support a new configuration, `spark.deploy.recoveryTimeout`.

### Why are the changes needed?

To allow the users to control this independently from 
`spark.worker.timeout`.

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

No.

### How was this patch tested?

Manual review.

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

No.

Closes #44282 from dongjoon-hyun/SPARK-46348.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala  | 4 +++-
 .../src/main/scala/org/apache/spark/internal/config/Deploy.scala | 9 +
 docs/spark-standalone.md | 9 +
 3 files changed, 21 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index be787dd29f8..7346c80aff4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -65,6 +65,8 @@ private[deploy] class Master(
   private val retainedDrivers = conf.get(RETAINED_DRIVERS)
   private val maxDrivers = conf.get(MAX_DRIVERS)
   private val reaperIterations = conf.get(REAPER_ITERATIONS)
+  private val recoveryTimeoutMs =
+conf.get(RECOVERY_TIMEOUT).map(_ * 1000).getOrElse(workerTimeoutMs)
   private val recoveryMode = conf.get(RECOVERY_MODE)
   private val maxExecutorRetries = conf.get(MAX_EXECUTOR_RETRIES)
 
@@ -246,7 +248,7 @@ private[deploy] class Master(
   override def run(): Unit = Utils.tryLogNonFatalError {
 self.send(CompleteRecovery)
   }
-}, workerTimeoutMs, TimeUnit.MILLISECONDS)
+}, recoveryTimeoutMs, TimeUnit.MILLISECONDS)
   }
 
 case CompleteRecovery => completeRecovery()
diff --git a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala 
b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala
index b52ea356789..6585d62b3b9 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.internal.config
 
 import java.util.Locale
+import java.util.concurrent.TimeUnit
 
 private[spark] object Deploy {
   val RECOVERY_MODE = ConfigBuilder("spark.deploy.recoveryMode")
@@ -56,6 +57,14 @@ private[spark] object Deploy {
 .stringConf
 .createWithDefault("")
 
+  val RECOVERY_TIMEOUT = ConfigBuilder("spark.deploy.recoveryTimeout")
+.doc("Configures the timeout for recovery process. The default value is 
the same " +
+  "with ${WORKER_TIMEOUT.key}.")
+.version("4.0.0")
+.timeConf(TimeUnit.SECONDS)
+.checkValue(_ > 0, "spark.deploy.recoveryTimeout must be positive.")
+.createOptional
+
   val ZOOKEEPER_URL = ConfigBuilder("spark.deploy.zookeeper.url")
 .doc(s"When `${RECOVERY_MODE.key}` is set to ZOOKEEPER, this " +
   "configuration is used to set the zookeeper URL to connect to.")
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 19935307221..0bc73978570 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -793,6 +793,15 @@ In order to enable this recovery mode, you can set 
SPARK_DAEMON_JAVA_OPTS in spa
 A compression codec for persistence engines. none (default), lz4, lzf, 
snappy, and zstd. Currently, only FILESYSTEM mode supports this 
configuration.
 4.0.0
   
+  
+spark.deploy.recoveryTimeout
+(none)
+
+  The timeout for recovery process. The default value is the same with
+  spark.worker.timeout.
+
+4.0.0
+  
   
 spark.deploy.recoveryMode.factory
 ""


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



(spark) branch master updated: [SPARK-46344][CORE] Warn properly when a driver exists successfully but master is disconnected

2023-12-09 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 166df8b1df59 [SPARK-46344][CORE] Warn properly when a driver exists 
successfully but master is disconnected
166df8b1df59 is described below

commit 166df8b1df5965fe3f15fecd5574545746b0b18f
Author: Dongjoon Hyun 
AuthorDate: Sat Dec 9 15:33:48 2023 -0800

[SPARK-46344][CORE] Warn properly when a driver exists successfully but 
master is disconnected

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

This PR aims to warn properly when a driver exists successfully but master 
is disconnected.

### Why are the changes needed?

In this case, `Master` considers them `Error` eventually.

![Screenshot 2023-12-09 at 3 05 27 
PM](https://github.com/apache/spark/assets/9700541/1323819b-4a0c-466d-afaa-845f507a905e)

**Worker Log**
```
23/12/09 15:13:21 INFO Worker: Driver driver-20231209151301-0003 exited 
successfully
=== Master is disconnected here ===
23/12/09 15:13:53 WARN Worker: Driver driver-20231209151332-0004 exited 
successfully while master is disconnected.
=== A new master starts and is connected here ===
23/12/09 15:17:10 INFO Worker: Driver driver-20231209151707- exited 
successfully
```

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

No.

### How was this patch tested?

Pass the CIs.

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

No.

Closes #44278 from dongjoon-hyun/SPARK-46344.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala 
b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 1422a1484f8d..785129e1d818 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -878,7 +878,12 @@ private[deploy] class Worker(
   case DriverState.FAILED =>
 logWarning(s"Driver $driverId exited with failure")
   case DriverState.FINISHED =>
-logInfo(s"Driver $driverId exited successfully")
+registrationRetryTimer match {
+  case Some(_) =>
+logWarning(s"Driver $driverId exited successfully while master is 
disconnected.")
+  case _ =>
+logInfo(s"Driver $driverId exited successfully")
+}
   case DriverState.KILLED =>
 logInfo(s"Driver $driverId was killed by user")
   case _ =>


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



(spark) branch branch-3.4 updated: [SPARK-46339][SS] Directory with batch number name should not be treated as metadata log

2023-12-09 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 4e80b3a09407 [SPARK-46339][SS] Directory with batch number name should 
not be treated as metadata log
4e80b3a09407 is described below

commit 4e80b3a09407042f7c596963dcb4fc59e68755ab
Author: Liang-Chi Hsieh 
AuthorDate: Sat Dec 9 15:20:55 2023 -0800

[SPARK-46339][SS] Directory with batch number name should not be treated as 
metadata log

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

This patch updates the document of `CheckpointFileManager.list` method to 
reflect the fact it is used to return both files and directories to reduce 
confusion.

For the usage like `HDFSMetadataLog` where it assumes returned file status 
by `list` are all files, we add a filter there to avoid confusing error.

### Why are the changes needed?

`HDFSMetadataLog` takes a metadata path as parameter. When it goes to 
retrieves all batches metadata, it calls `CheckpointFileManager.list` to get 
all files under the metadata path. However, currently all implementations of 
`CheckpointFileManager.list` returns all files/directories under the given 
path. So if there is a dictionary with name of batch number (a long value), the 
directory will be returned too and cause trouble when `HDFSMetadataLog` goes to 
read it.

Actually, `CheckpointFileManager.list` method clearly defines that it lists 
the "files" in a path. That's being said, current implementations don't follow 
the doc. We tried to make `list` method implementations only return files but 
some usage (state metadata) of `list` method already break the assumption and 
they use dictionaries returned by `list` method. So we simply update `list` 
method document to explicitly define it returns both files/dictionaries. We add 
a filter in `HDFSMetad [...]

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

No

### How was this patch tested?

Added test

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

No

Closes #44272 from viirya/fix_metadatalog.

Authored-by: Liang-Chi Hsieh 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 75805f07f5caeb01104a7352b02790d03a043ded)
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 28a8b181e96d4ce71e2f9888910214d14a859b7d)
Signed-off-by: Dongjoon Hyun 
---
 .../sql/execution/streaming/CheckpointFileManager.scala  |  4 ++--
 .../spark/sql/execution/streaming/HDFSMetadataLog.scala  |  2 ++
 .../spark/sql/execution/streaming/HDFSMetadataLogSuite.scala | 12 
 3 files changed, 16 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
index 013efd3c7bae..b2a3b8d73d4e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
@@ -65,10 +65,10 @@ trait CheckpointFileManager {
   /** Open a file for reading, or throw exception if it does not exist. */
   def open(path: Path): FSDataInputStream
 
-  /** List the files in a path that match a filter. */
+  /** List the files/directories in a path that match a filter. */
   def list(path: Path, filter: PathFilter): Array[FileStatus]
 
-  /** List all the files in a path. */
+  /** List all the files/directories in a path. */
   def list(path: Path): Array[FileStatus] = {
 list(path, (_: Path) => true)
   }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
index 2b0172bb9555..9a811db679d0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
@@ -325,6 +325,8 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: 
SparkSession, path:
   /** List the available batches on file system. */
   protected def listBatches: Array[Long] = {
 val batchIds = fileManager.list(metadataPath, batchFilesFilter)
+  // Batches must be files
+  .filter(f => f.isFile)
   .map(f => pathToBatchId(f.getPath)) ++
   // Iterate over keySet is not thread safe. We call `toArray` to make a 
copy in the lock to
   // elimiate the race condition.
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFS

(spark) branch branch-3.5 updated: [SPARK-46339][SS] Directory with batch number name should not be treated as metadata log

2023-12-09 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 28a8b181e96d [SPARK-46339][SS] Directory with batch number name should 
not be treated as metadata log
28a8b181e96d is described below

commit 28a8b181e96d4ce71e2f9888910214d14a859b7d
Author: Liang-Chi Hsieh 
AuthorDate: Sat Dec 9 15:20:55 2023 -0800

[SPARK-46339][SS] Directory with batch number name should not be treated as 
metadata log

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

This patch updates the document of `CheckpointFileManager.list` method to 
reflect the fact it is used to return both files and directories to reduce 
confusion.

For the usage like `HDFSMetadataLog` where it assumes returned file status 
by `list` are all files, we add a filter there to avoid confusing error.

### Why are the changes needed?

`HDFSMetadataLog` takes a metadata path as parameter. When it goes to 
retrieves all batches metadata, it calls `CheckpointFileManager.list` to get 
all files under the metadata path. However, currently all implementations of 
`CheckpointFileManager.list` returns all files/directories under the given 
path. So if there is a dictionary with name of batch number (a long value), the 
directory will be returned too and cause trouble when `HDFSMetadataLog` goes to 
read it.

Actually, `CheckpointFileManager.list` method clearly defines that it lists 
the "files" in a path. That's being said, current implementations don't follow 
the doc. We tried to make `list` method implementations only return files but 
some usage (state metadata) of `list` method already break the assumption and 
they use dictionaries returned by `list` method. So we simply update `list` 
method document to explicitly define it returns both files/dictionaries. We add 
a filter in `HDFSMetad [...]

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

No

### How was this patch tested?

Added test

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

No

Closes #44272 from viirya/fix_metadatalog.

Authored-by: Liang-Chi Hsieh 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 75805f07f5caeb01104a7352b02790d03a043ded)
Signed-off-by: Dongjoon Hyun 
---
 .../sql/execution/streaming/CheckpointFileManager.scala  |  4 ++--
 .../spark/sql/execution/streaming/HDFSMetadataLog.scala  |  2 ++
 .../spark/sql/execution/streaming/HDFSMetadataLogSuite.scala | 12 
 3 files changed, 16 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
index ad3212871fc9..677e2fccb6b4 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
@@ -65,10 +65,10 @@ trait CheckpointFileManager {
   /** Open a file for reading, or throw exception if it does not exist. */
   def open(path: Path): FSDataInputStream
 
-  /** List the files in a path that match a filter. */
+  /** List the files/directories in a path that match a filter. */
   def list(path: Path, filter: PathFilter): Array[FileStatus]
 
-  /** List all the files in a path. */
+  /** List all the files/directories in a path. */
   def list(path: Path): Array[FileStatus] = {
 list(path, (_: Path) => true)
   }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
index 2b0172bb9555..9a811db679d0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
@@ -325,6 +325,8 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: 
SparkSession, path:
   /** List the available batches on file system. */
   protected def listBatches: Array[Long] = {
 val batchIds = fileManager.list(metadataPath, batchFilesFilter)
+  // Batches must be files
+  .filter(f => f.isFile)
   .map(f => pathToBatchId(f.getPath)) ++
   // Iterate over keySet is not thread safe. We call `toArray` to make a 
copy in the lock to
   // elimiate the race condition.
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
index 980d532dd477..08f245135f58 100644
--- 
a/sql/core/src/test/scala/org/apac

(spark) branch master updated: [SPARK-46339][SS] Directory with batch number name should not be treated as metadata log

2023-12-09 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 75805f07f5ca [SPARK-46339][SS] Directory with batch number name should 
not be treated as metadata log
75805f07f5ca is described below

commit 75805f07f5caeb01104a7352b02790d03a043ded
Author: Liang-Chi Hsieh 
AuthorDate: Sat Dec 9 15:20:55 2023 -0800

[SPARK-46339][SS] Directory with batch number name should not be treated as 
metadata log

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

This patch updates the document of `CheckpointFileManager.list` method to 
reflect the fact it is used to return both files and directories to reduce 
confusion.

For the usage like `HDFSMetadataLog` where it assumes returned file status 
by `list` are all files, we add a filter there to avoid confusing error.

### Why are the changes needed?

`HDFSMetadataLog` takes a metadata path as parameter. When it goes to 
retrieves all batches metadata, it calls `CheckpointFileManager.list` to get 
all files under the metadata path. However, currently all implementations of 
`CheckpointFileManager.list` returns all files/directories under the given 
path. So if there is a dictionary with name of batch number (a long value), the 
directory will be returned too and cause trouble when `HDFSMetadataLog` goes to 
read it.

Actually, `CheckpointFileManager.list` method clearly defines that it lists 
the "files" in a path. That's being said, current implementations don't follow 
the doc. We tried to make `list` method implementations only return files but 
some usage (state metadata) of `list` method already break the assumption and 
they use dictionaries returned by `list` method. So we simply update `list` 
method document to explicitly define it returns both files/dictionaries. We add 
a filter in `HDFSMetad [...]

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

No

### How was this patch tested?

Added test

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

No

Closes #44272 from viirya/fix_metadatalog.

Authored-by: Liang-Chi Hsieh 
Signed-off-by: Dongjoon Hyun 
---
 .../sql/execution/streaming/CheckpointFileManager.scala  |  4 ++--
 .../spark/sql/execution/streaming/HDFSMetadataLog.scala  |  2 ++
 .../spark/sql/execution/streaming/HDFSMetadataLogSuite.scala | 12 
 3 files changed, 16 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
index af2c97b21138..34c5dee0997b 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala
@@ -65,10 +65,10 @@ trait CheckpointFileManager {
   /** Open a file for reading, or throw exception if it does not exist. */
   def open(path: Path): FSDataInputStream
 
-  /** List the files in a path that match a filter. */
+  /** List the files/directories in a path that match a filter. */
   def list(path: Path, filter: PathFilter): Array[FileStatus]
 
-  /** List all the files in a path. */
+  /** List all the files/directories in a path. */
   def list(path: Path): Array[FileStatus] = {
 list(path, (_: Path) => true)
   }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
index 79627030e1eb..b3eedbf93f04 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala
@@ -327,6 +327,8 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: 
SparkSession, path:
   /** List the available batches on file system. */
   protected def listBatches: Array[Long] = {
 val batchIds = fileManager.list(metadataPath, batchFilesFilter)
+  // Batches must be files
+  .filter(f => f.isFile)
   .map(f => pathToBatchId(f.getPath)) ++
   // Iterate over keySet is not thread safe. We call `toArray` to make a 
copy in the lock to
   // elimiate the race condition.
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
index 980d532dd477..08f245135f58 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/s

(spark) branch master updated (1af4b658064d -> cae4bdc88161)

2023-12-09 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 1af4b658064d [SPARK-46338][PS][TESTS] Re-enable the `get_item` test 
for `BasicIndexingTests`
 add cae4bdc88161 [SPARK-46322][PYTHON][DOCS] Replace external link with 
internal link for error documentation

No new revisions were added by this update.

Summary of changes:
 python/pyspark/errors_doc_gen.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)


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



(spark) branch master updated: [SPARK-46338][PS][TESTS] Re-enable the `get_item` test for `BasicIndexingTests`

2023-12-09 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 1af4b658064d [SPARK-46338][PS][TESTS] Re-enable the `get_item` test 
for `BasicIndexingTests`
1af4b658064d is described below

commit 1af4b658064d05b15c4a8409a8aa13df63b64ca4
Author: Haejoon Lee 
AuthorDate: Sat Dec 9 13:46:13 2023 -0800

[SPARK-46338][PS][TESTS] Re-enable the `get_item` test for 
`BasicIndexingTests`

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

This PR proposes to re-enable the `get_item` test for `BasicIndexingTests`.

### Why are the changes needed?

To improve the test coverage.

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

No.

### How was this patch tested?

Enable the test.

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

Closes #44271 from itholic/enable_test_getitem.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/pandas/tests/test_indexing.py | 12 ++--
 1 file changed, 6 insertions(+), 6 deletions(-)

diff --git a/python/pyspark/pandas/tests/test_indexing.py 
b/python/pyspark/pandas/tests/test_indexing.py
index a4ca03005b33..eb86c9ffabc4 100644
--- a/python/pyspark/pandas/tests/test_indexing.py
+++ b/python/pyspark/pandas/tests/test_indexing.py
@@ -742,13 +742,13 @@ class IndexingTest(ComparisonTestBase):
 self.assertRaises(AttributeError, lambda: psdf.X)
 
 # not str/unicode
-# TODO?: pdf = pd.DataFrame(np.random.randn(10, 5))
-# TODO?: psdf = ps.from_pandas(pdf)
-# TODO?: self.assert_eq(psdf[0], pdf[0])
-# TODO?: self.assert_eq(psdf[[1, 2]], pdf[[1, 2]])
+pdf = pd.DataFrame(np.random.randn(10, 5))
+psdf = ps.from_pandas(pdf)
+self.assert_eq(psdf[0], pdf[0])
+self.assert_eq(psdf[[1, 2]], pdf[[1, 2]])
 
-# TODO?: self.assertRaises(KeyError, lambda: pdf[8])
-# TODO?: self.assertRaises(KeyError, lambda: pdf[[1, 8]])
+self.assertRaises(KeyError, lambda: pdf[8])
+self.assertRaises(KeyError, lambda: pdf[[1, 8]])
 
 # non-string column names
 pdf = pd.DataFrame(


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



(spark) branch master updated: [SPARK-46334][INFRA][PS] Upgrade `Pandas` to 2.1.4

2023-12-08 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 48e773451eeb [SPARK-46334][INFRA][PS] Upgrade `Pandas` to 2.1.4
48e773451eeb is described below

commit 48e773451eeb7feb84b3f11883bd76217f2233c4
Author: Bjørn Jørgensen 
AuthorDate: Fri Dec 8 18:39:10 2023 -0800

[SPARK-46334][INFRA][PS] Upgrade `Pandas` to 2.1.4

### What changes were proposed in this pull request?
Upgrade pandas from 2.1.3 to 2.1.4

### Why are the changes needed?
[Release notes](https://pandas.pydata.org/docs/whatsnew/v2.1.4.html)

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

### How was this patch tested?
Pass GA

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

Closes #44266 from bjornjorgensen/pandas2.1.4.

Authored-by: Bjørn Jørgensen 
Signed-off-by: Dongjoon Hyun 
---
 dev/infra/Dockerfile   | 4 ++--
 python/pyspark/pandas/supported_api_gen.py | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile
index 3e449bcb6c82..cade845d911b 100644
--- a/dev/infra/Dockerfile
+++ b/dev/infra/Dockerfile
@@ -92,10 +92,10 @@ RUN mkdir -p /usr/local/pypy/pypy3.8 && \
 ln -sf /usr/local/pypy/pypy3.8/bin/pypy /usr/local/bin/pypy3.8 && \
 ln -sf /usr/local/pypy/pypy3.8/bin/pypy /usr/local/bin/pypy3
 RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3
-RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas<=2.1.3' scipy coverage 
matplotlib
+RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas<=2.1.4' scipy coverage 
matplotlib
 
 
-ARG BASIC_PIP_PKGS="numpy pyarrow>=14.0.0 six==1.16.0 pandas<=2.1.3 scipy 
unittest-xml-reporting plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl 
memory-profiler>=0.61.0 scikit-learn>=1.3.2"
+ARG BASIC_PIP_PKGS="numpy pyarrow>=14.0.0 six==1.16.0 pandas<=2.1.4 scipy 
unittest-xml-reporting plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl 
memory-profiler>=0.61.0 scikit-learn>=1.3.2"
 # Python deps for Spark Connect
 ARG CONNECT_PIP_PKGS="grpcio==1.59.3 grpcio-status==1.59.3 protobuf==4.25.1 
googleapis-common-protos==1.56.4"
 
diff --git a/python/pyspark/pandas/supported_api_gen.py 
b/python/pyspark/pandas/supported_api_gen.py
index 1f893520d2ce..a598fc816d96 100644
--- a/python/pyspark/pandas/supported_api_gen.py
+++ b/python/pyspark/pandas/supported_api_gen.py
@@ -37,7 +37,7 @@ from pyspark.pandas.exceptions import 
PandasNotImplementedError
 MAX_MISSING_PARAMS_SIZE = 5
 COMMON_PARAMETER_SET = {"kwargs", "args", "cls"}
 MODULE_GROUP_MATCH = [(pd, ps), (pdw, psw), (pdg, psg)]
-PANDAS_LATEST_VERSION = "2.1.3"
+PANDAS_LATEST_VERSION = "2.1.4"
 
 RST_HEADER = """
 =


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



(spark) branch branch-3.4 updated: [SPARK-46275][3.4] Protobuf: Return null in permissive mode when deserialization fails

2023-12-08 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 4745138601b7 [SPARK-46275][3.4] Protobuf: Return null in permissive 
mode when deserialization fails
4745138601b7 is described below

commit 4745138601b74e805459bd240f748fcf3e7ddec2
Author: Raghu Angadi 
AuthorDate: Fri Dec 8 14:40:03 2023 -0800

[SPARK-46275][3.4] Protobuf: Return null in permissive mode when 
deserialization fails

This is a cherry-pick of #44214 into 3.4 branch.

From the original PR:

### What changes were proposed in this pull request?
This updates the the behavior of `from_protobuf()` built function when 
underlying record fails to deserialize.

  * **Current behvior**:
* By default, this would throw an error and the query fails. [This part 
is not changed in the PR]
* When `mode` is set to 'PERMISSIVE' it returns a non-null struct with 
each of the inner fields set to null e.g. `{ "field_a": null, "field_b": null 
}`  etc.
   * This is not very convenient to the users. They don't know if this 
was due to malformed record or if the input itself has null. It is very hard to 
check for each field for null in SQL query (imagine a sql query with a struct 
that has 10 fields).

  * **New behavior**
* When `mode` is set to 'PERMISSIVE' it simply returns `null`.

### Why are the changes needed?
This makes it easier for users to detect and handle malformed records.

### Does this PR introduce _any_ user-facing change?
Yes, but this does not change the contract. In fact, it clarifies it.

### How was this patch tested?
 - Unit tests are updated.

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

Closes #44265 from rangadi/protobuf-null-3.4.

Authored-by: Raghu Angadi 
Signed-off-by: Dongjoon Hyun 
---
 .../sql/protobuf/ProtobufDataToCatalyst.scala  | 31 --
 .../ProtobufCatalystDataConversionSuite.scala  | 13 +
 2 files changed, 6 insertions(+), 38 deletions(-)

diff --git 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala
 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala
index da44f94d5eac..78e995190045 100644
--- 
a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala
+++ 
b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufDataToCatalyst.scala
@@ -21,12 +21,12 @@ import scala.util.control.NonFatal
 
 import com.google.protobuf.DynamicMessage
 
-import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, 
Expression, SpecificInternalRow, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, 
Expression, UnaryExpression}
 import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, 
CodeGenerator, ExprCode}
 import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, 
PermissiveMode}
 import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
 import org.apache.spark.sql.protobuf.utils.{ProtobufOptions, ProtobufUtils, 
SchemaConverters}
-import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType, 
StructType}
+import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType}
 
 private[protobuf] case class ProtobufDataToCatalyst(
 child: Expression,
@@ -38,16 +38,8 @@ private[protobuf] case class ProtobufDataToCatalyst(
 
   override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)
 
-  override lazy val dataType: DataType = {
-val dt = SchemaConverters.toSqlType(messageDescriptor, 
protobufOptions).dataType
-parseMode match {
-  // With PermissiveMode, the output Catalyst row might contain columns of 
null values for
-  // corrupt records, even if some of the columns are not nullable in the 
user-provided schema.
-  // Therefore we force the schema to be all nullable here.
-  case PermissiveMode => dt.asNullable
-  case _ => dt
-}
-  }
+  override lazy val dataType: DataType =
+SchemaConverters.toSqlType(messageDescriptor, protobufOptions).dataType
 
   override def nullable: Boolean = true
 
@@ -75,22 +67,9 @@ private[protobuf] case class ProtobufDataToCatalyst(
 mode
   }
 
-  @transient private lazy val nullResultRow: Any = dataType match {
-case st: StructType =>
-  val resultRow = new SpecificInternalRow(st.map(_.dataType))
-  for (i <- 0 until st.length) {
-resultRow.setNullAt(i)
-  }
-  resultRow
-
-case _ =>
-  null
-  }
-
   private def handleException(e: Throwable): Any = {
 parseMode match {
-  case PermissiveMode =&

(spark) branch master updated: [SPARK-46325][CONNECT] Remove unnecessary override functions when constructing `WrappedCloseableIterator` in `ResponseValidator#wrapIterator`

2023-12-08 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 75cb37011fa2 [SPARK-46325][CONNECT] Remove unnecessary override 
functions when constructing `WrappedCloseableIterator` in 
`ResponseValidator#wrapIterator`
75cb37011fa2 is described below

commit 75cb37011fa2a1cb00e633fac312eb15ab412df9
Author: yangjie01 
AuthorDate: Fri Dec 8 14:00:28 2023 -0800

[SPARK-46325][CONNECT] Remove unnecessary override functions when 
constructing `WrappedCloseableIterator` in `ResponseValidator#wrapIterator`

### What changes were proposed in this pull request?
This pr removes the overridden `hasNext` and `close` functions in the 
construction of `WrappedCloseableIterator` in `ResponseValidator#wrapIterator`, 
as these functions are identical to those defined in `WrappedCloseableIterator`.

- WrappedCloseableIterator


https://github.com/apache/spark/blob/9ffdcc398ed5560f34778d005da697f6ad0a15ee/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala#L30-L42

- ResponseValidator#wrapIterator


https://github.com/apache/spark/blob/9ffdcc398ed5560f34778d005da697f6ad0a15ee/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala#L62-L85

### Why are the changes needed?
Remove unnecessary override functions.

### 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 #44255 from LuciferYang/SPARK-46325.

Authored-by: yangjie01 
Signed-off-by: Dongjoon Hyun 
---
 .../apache/spark/sql/connect/client/ResponseValidator.scala   | 11 ---
 1 file changed, 11 deletions(-)

diff --git 
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala
 
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala
index 2081196d4671..67f29c727ef4 100644
--- 
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala
+++ 
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala
@@ -65,22 +65,11 @@ class ResponseValidator extends Logging {
 
   override def innerIterator: Iterator[T] = inner
 
-  override def hasNext: Boolean = {
-innerIterator.hasNext
-  }
-
   override def next(): T = {
 verifyResponse {
   innerIterator.next()
 }
   }
-
-  override def close(): Unit = {
-innerIterator match {
-  case it: CloseableIterator[T] => it.close()
-  case _ => // nothing
-}
-  }
 }
   }
 


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



(spark) branch master updated: [SPARK-46332][SQL] Migrate `CatalogNotFoundException` to the error class `CATALOG_NOT_FOUND`

2023-12-08 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 d9f0fccd967b [SPARK-46332][SQL] Migrate `CatalogNotFoundException` to 
the error class `CATALOG_NOT_FOUND`
d9f0fccd967b is described below

commit d9f0fccd967b5c8686353d524d2b31e27b7a473b
Author: Max Gekk 
AuthorDate: Fri Dec 8 12:54:20 2023 -0800

[SPARK-46332][SQL] Migrate `CatalogNotFoundException` to the error class 
`CATALOG_NOT_FOUND`

### What changes were proposed in this pull request?
In the PR, I propose to migrate the `CatalogNotFoundException` exception to 
the new error class `CATALOG_NOT_FOUND`, improve the format of the exception 
message, and prohibit creation of the exception without the error class.

### Why are the changes needed?
This is a part of the migration process onto error classes and new error 
framework. The changes improve user experience w/ Spark SQL, and make 
`CatalogNotFoundException` consistent to other Spark exceptions.

### Does this PR introduce _any_ user-facing change?
Yes, if user's code depends on the error message format of 
`CatalogNotFoundException`.

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt "core/testOnly *SparkThrowableSuite"
```

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

Closes #44259 from MaxGekk/catalog-plugin-not-found.

Authored-by: Max Gekk 
Signed-off-by: Dongjoon Hyun 
---
 R/pkg/tests/fulltests/test_sparkSQL.R  |  5 +
 common/utils/src/main/resources/error/error-classes.json   |  6 ++
 .../jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala |  6 +++---
 docs/sql-error-conditions.md   |  6 ++
 .../spark/sql/connector/catalog/CatalogNotFoundException.scala | 10 +++---
 .../org/apache/spark/sql/connector/catalog/Catalogs.scala  |  2 +-
 .../org/apache/spark/sql/errors/QueryExecutionErrors.scala |  7 +--
 .../spark/sql/connector/catalog/CatalogLoadingSuite.java   |  7 ++-
 .../spark/sql/catalyst/analysis/TableLookupCacheSuite.scala|  6 +++---
 .../spark/sql/connector/catalog/LookupCatalogSuite.scala   |  5 +++--
 .../org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala  |  6 --
 .../sql/execution/command/AlignAssignmentsSuiteBase.scala  |  5 +++--
 .../spark/sql/execution/command/PlanResolutionSuite.scala  |  9 -
 13 files changed, 48 insertions(+), 32 deletions(-)

diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R 
b/R/pkg/tests/fulltests/test_sparkSQL.R
index f2bef7a00446..0d96f708a544 100644
--- a/R/pkg/tests/fulltests/test_sparkSQL.R
+++ b/R/pkg/tests/fulltests/test_sparkSQL.R
@@ -4103,10 +4103,7 @@ test_that("catalog APIs, listCatalogs, 
setCurrentCatalog, currentCatalog", {
   expect_equal(currentCatalog(), "spark_catalog")
   expect_error(setCurrentCatalog("spark_catalog"), NA)
   expect_error(setCurrentCatalog("zxwtyswklpf"),
-   paste0("Error in setCurrentCatalog : ",
-   
"org.apache.spark.sql.connector.catalog.CatalogNotFoundException: ",
-   "Catalog 'zxwtyswklpf' plugin class not found: ",
-   "spark.sql.catalog.zxwtyswklpf is not defined"))
+   "[CATALOG_NOT_FOUND]*`zxwtyswklpf`*")
   catalogs <- collect(listCatalogs())
 })
 
diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 7a672fa5e557..62d10c0d34cb 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -407,6 +407,12 @@
 ],
 "sqlState" : "22003"
   },
+  "CATALOG_NOT_FOUND" : {
+"message" : [
+  "The catalog  not found. Consider to set the SQL config 
 to a catalog plugin."
+],
+"sqlState" : "42P08"
+  },
   "CHECKPOINT_RDD_BLOCK_ID_NOT_FOUND" : {
 "message" : [
   "Checkpoint block  not found!",
diff --git 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala
 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala
index cefa63ecd353..d646fad00c07 100644
--- 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala
+++ 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala
@@ -66,10 +66,10 @@ class CatalogSuite extends RemoteSparkSession with 
SQLHelper {
   val catalogs = spark.catalog.listCatalogs().collect()
 

(spark) branch master updated: [SPARK-46324][SQL][PYTHON] Fix the output name of pyspark.sql.functions.user and session_user

2023-12-08 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 3224cddcf9da [SPARK-46324][SQL][PYTHON] Fix the output name of 
pyspark.sql.functions.user and session_user
3224cddcf9da is described below

commit 3224cddcf9da913c964b775b5912a67cd1e968b2
Author: Hyukjin Kwon 
AuthorDate: Fri Dec 8 12:48:24 2023 -0800

[SPARK-46324][SQL][PYTHON] Fix the output name of 
pyspark.sql.functions.user and session_user

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

This PR proposes to fix `user()` and `session_user()` to have the same 
names in its output name.

### Why are the changes needed?

To show the correct name of the functions being used.

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

Yes.

```scala
spark.range(1).select(user(), session_user()).show()
```

Before:

```
+--+--+
|current_user()|current_user()|
+--+--+
|  hyukjin.kwon|  hyukjin.kwon|
+--+--+
```

After:

```
+--+--+
|user()|session_user()|
+--+--+
|  hyukjin.kwon|  hyukjin.kwon|
+--+--+
```

### How was this patch tested?

Manually tested, and unittests were added.

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

No.

Closes #44253 from HyukjinKwon/user-name.

Authored-by: Hyukjin Kwon 
Signed-off-by: Dongjoon Hyun 
---
 .../query-tests/explain-results/function_session_user.explain| 2 +-
 .../resources/query-tests/explain-results/function_user.explain  | 2 +-
 python/pyspark/sql/functions/builtin.py  | 4 ++--
 python/pyspark/sql/tests/test_functions.py   | 9 +
 .../scala/org/apache/spark/sql/catalyst/expressions/misc.scala   | 5 +++--
 .../src/test/resources/sql-functions/sql-expression-schema.md| 6 +++---
 6 files changed, 19 insertions(+), 9 deletions(-)

diff --git 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_session_user.explain
 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_session_user.explain
index 82f5d2adcec0..b6205d9fb56c 100644
--- 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_session_user.explain
+++ 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_session_user.explain
@@ -1,2 +1,2 @@
-Project [current_user() AS current_user()#0]
+Project [session_user() AS session_user()#0]
 +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_user.explain
 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_user.explain
index 82f5d2adcec0..52746c58c000 100644
--- 
a/connector/connect/common/src/test/resources/query-tests/explain-results/function_user.explain
+++ 
b/connector/connect/common/src/test/resources/query-tests/explain-results/function_user.explain
@@ -1,2 +1,2 @@
-Project [current_user() AS current_user()#0]
+Project [user() AS user()#0]
 +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git a/python/pyspark/sql/functions/builtin.py 
b/python/pyspark/sql/functions/builtin.py
index 997b641080cf..e1c01018 100644
--- a/python/pyspark/sql/functions/builtin.py
+++ b/python/pyspark/sql/functions/builtin.py
@@ -8967,7 +8967,7 @@ def user() -> Column:
 >>> import pyspark.sql.functions as sf
 >>> spark.range(1).select(sf.user()).show() # doctest: +SKIP
 +--+
-|current_user()|
+|user()|
 +--+
 | ruifeng.zheng|
 +--+
@@ -8986,7 +8986,7 @@ def session_user() -> Column:
 >>> import pyspark.sql.functions as sf
 >>> spark.range(1).select(sf.session_user()).show() # doctest: +SKIP
 +--+
-|current_user()|
+|session_user()|
 +--+
 | ruifeng.zheng|
 +--+
diff --git a/python/pyspark/sql/tests/test_functions.py 
b/python/pyspark/sql/tests/test_functions.py
index b59417d8a310..5352ee04d7fe 100644
--- a/python/pyspark/sql/tests/test_functions.py
+++ b/python/pyspark/sql/tests/test_functions.py
@@ -1355,6 +1355,15 @@ class FunctionsTestsMixin:
 message_parameters={"arg_name": "gapDuration", "arg_type": "int"},
 )
 
+def test_current_user(self):
+df = self.spark.range(1).select(F.current_user())
+self.assertIsInstance(df.first()[0], str)
+self.assertEqual(df.

svn commit: r65952 - in /dev/spark/v3.3.4-rc1-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/R/articles/ _site/api/R/deps/ _site/api/R/deps/bootstrap-5.3.1/ _site/api/R/deps/bootstrap-5.3.1/fonts/

2023-12-08 Thread dongjoon
Author: dongjoon
Date: Fri Dec  8 20:21:58 2023
New Revision: 65952

Log:
Apache Spark v3.3.4-rc1 docs


[This commit notification would consist of 2683 parts, 
which exceeds the limit of 50 ones, so it was shortened to the summary.]

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



svn commit: r65951 - /dev/spark/v3.4.2-rc1-docs/

2023-12-08 Thread dongjoon
Author: dongjoon
Date: Fri Dec  8 19:49:07 2023
New Revision: 65951

Log:
Remove Apache Spark 3.4.2 RC1 docs after 3.4.2 release

Removed:
dev/spark/v3.4.2-rc1-docs/


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



svn commit: r65950 - /dev/spark/v3.3.4-rc1-bin/

2023-12-08 Thread dongjoon
Author: dongjoon
Date: Fri Dec  8 19:47:54 2023
New Revision: 65950

Log:
Apache Spark v3.3.4-rc1

Added:
dev/spark/v3.3.4-rc1-bin/
dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz   (with props)
dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz.asc
dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz.sha512
dev/spark/v3.3.4-rc1-bin/pyspark-3.3.4.tar.gz   (with props)
dev/spark/v3.3.4-rc1-bin/pyspark-3.3.4.tar.gz.asc
dev/spark/v3.3.4-rc1-bin/pyspark-3.3.4.tar.gz.sha512
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop2.tgz   (with props)
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop2.tgz.asc
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop2.tgz.sha512
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop3-scala2.13.tgz   (with 
props)
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop3-scala2.13.tgz.asc
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop3-scala2.13.tgz.sha512
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop3.tgz   (with props)
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop3.tgz.asc
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-hadoop3.tgz.sha512
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-without-hadoop.tgz   (with props)
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-without-hadoop.tgz.asc
dev/spark/v3.3.4-rc1-bin/spark-3.3.4-bin-without-hadoop.tgz.sha512
dev/spark/v3.3.4-rc1-bin/spark-3.3.4.tgz   (with props)
dev/spark/v3.3.4-rc1-bin/spark-3.3.4.tgz.asc
dev/spark/v3.3.4-rc1-bin/spark-3.3.4.tgz.sha512

Added: dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz.asc
==
--- dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz.asc (added)
+++ dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz.asc Fri Dec  8 19:47:54 2023
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJIBAABCgAyFiEE8oycklwYjDXjRWFN7aAM6DTw/FwFAmVzcocUHGRvbmdqb29u
+QGFwYWNoZS5vcmcACgkQ7aAM6DTw/FxycA//Vd2YOoAE4QBm3xcAVuHdGesXVBhG
+Ja7TIc2WLkxi4P/wRKi6vYtacgMuayygetDqIDNOv8w/xk20omwLvA7ComVv2kbz
+abIyWOZggUOhOumYY5FyIskEZBygCiTgWqyZSSi13tj8DT2bUL931N4J/E+B8V+/
+tY77UvemmjNOxRvzmqDnZz3pp+faJ5J18UDMXG4J0atM7eLjT7vEpL4cAHUDuYGO
+YHvwTxrUJxFyGafNkAUrFdAQmHaW+ULKawHPTBfex/sdojUqkQ0oiC7Z6th2ZpZ4
+R+yrs0GxaMaMLseifiK7sy6t/cHmGNghAPo0bl+sVe1rY8NDN075eaHOXQ5I+WqL
+dNBarAEHJ7s50ClVNQtUIu4QWE7lydrW2ykRfKHAa0v5Qffe+vJGBmmv/RHRa2Wi
+u9mohwbu3i6q1SRJFz2tpSF1ZxeAPshAy+/TMH0BDznf7lNcca/dFZGmgWNL6B4S
+5YCbtK5JZ6JUjcVUSFi/1qQy2PfjDPmTYYDBqPNzx8M7kVXx2mbvWwslC0+KdnBK
+OpMiSCSR5p3jz9sekKAI96vtzX5Q/zJc/+0J+wpk8Gpe5K2ajOG5oAbH0WQn5ex4
+rZ3eh9NH+VH5szmA9ZoJe7vf9fpLk3I2PPAhAoZPRMnx048+/kyzmsisJ52h14Tj
+/b6FUTAqwOGzM8I=
+=b7JY
+-END PGP SIGNATURE-

Added: dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz.sha512
==
--- dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz.sha512 (added)
+++ dev/spark/v3.3.4-rc1-bin/SparkR_3.3.4.tar.gz.sha512 Fri Dec  8 19:47:54 2023
@@ -0,0 +1 @@
+33b8848b7b409302945e4ef91ff254507e5f3766ad19404473889ce9ce5d88c5fa9d6a806d1120c8f43c16137a76426183af43e974a2f0bb12cfefce566fb048
  SparkR_3.3.4.tar.gz

Added: dev/spark/v3.3.4-rc1-bin/pyspark-3.3.4.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.3.4-rc1-bin/pyspark-3.3.4.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.3.4-rc1-bin/pyspark-3.3.4.tar.gz.asc
==
--- dev/spark/v3.3.4-rc1-bin/pyspark-3.3.4.tar.gz.asc (added)
+++ dev/spark/v3.3.4-rc1-bin/pyspark-3.3.4.tar.gz.asc Fri Dec  8 19:47:54 2023
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJIBAABCgAyFiEE8oycklwYjDXjRWFN7aAM6DTw/FwFAmVzcokUHGRvbmdqb29u
+QGFwYWNoZS5vcmcACgkQ7aAM6DTw/FwUPQ//ST9CejnIpZvrG8L6IEDk/G2QMPZR
+LaFUy3DVdZsJ6JBdbo4wQWYl51CO5OvUzYqJC0WYtSdkOS34FagSccwjSg3mzhIa
+FObQ86fdYk9wk+2fJl7+jTV6z1SZ99lRjLlN2JyFJLICm+SPVekCDMBmTDS/Ohrg
+AVm/yMwjrtSs2QzxlmUIF+8IrriFHQtKUENx1kb0aYWSHgarGqJvzB6UcBOzAter
+47vjj0nCfTSCafqyME2TZ6XVopOtrUK+ZENH9MXv1VyKfp4F6tLLMOf7ZyOa44S1
+PfEEOO+SllP9/nW30ydRtcD3ZsMT8YL6+sOuBt+y7nc8c6cJKVUjC62kAIA0yTga
+XJU8fu+vT6HGELjME3V0LAiOKZ/IIhICgJENZRHXPVFZPa+gWSBhTGQyZD5e0ujD
+iXr5Rrenp17fwAKtt1OeuJDkAht1J4Z/2zt6VzlqbM2GR0EbtDIbNl0TMaFoRkmH
+HzwSsY2AsKCks4Ze7DBp9S0Pji93cWybqndm/SDEIRXSKegRus/i+8SFJv03IlZr
+Mzkn7IW1RiM17SFd29q5hYyVyZ7Zff40dqYN7rkGWXQBul4U8vRI3WGeOrSB2Oce
+jEqgBA1gArm+iojSkAdiAlrDgudNZViTLg0kUtCcdYSb3qFFtRAWEEBXvAAbMeBU
+wSHbZ1fFgEELKNk=
+=KLJ9
+-END PGP SIGNATURE-

Added: dev/spark

(spark) branch master updated: [SPARK-46328][SQL] Allocate capacity of array list of TColumns by columns size in TRowSet generation

2023-12-08 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 20c9b3dc4fac [SPARK-46328][SQL] Allocate capacity of array list of 
TColumns by columns size in TRowSet generation
20c9b3dc4fac is described below

commit 20c9b3dc4fac283f895c8d860b4c6e0144697302
Author: liangbowen 
AuthorDate: Fri Dec 8 11:24:35 2023 -0800

[SPARK-46328][SQL] Allocate capacity of array list of TColumns by columns 
size in TRowSet generation

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

Allocate enough capacity by columns size for assembling array list of 
TColumns in TRowSet generation.

### Why are the changes needed?

ArrayLists is created for TColumn value collections in RowSetUtils for 
TRowSet generation. Currently, they are created with Java's default capacity of 
16, rather than by the number of columns, which could cause array copying in 
assembling each TColumn collection when the column number exceeds the default 
capacity.

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

No.

### How was this patch tested?

GA tests.

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

No.

Closes #44258 from bowenliang123/rowset-cap.

Authored-by: liangbowen 
Signed-off-by: Dongjoon Hyun 
---
 .../org/apache/spark/sql/hive/thriftserver/RowSetUtils.scala | 9 ++---
 1 file changed, 6 insertions(+), 3 deletions(-)

diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/RowSetUtils.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/RowSetUtils.scala
index 94046adca0d8..502e29619027 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/RowSetUtils.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/RowSetUtils.scala
@@ -57,15 +57,16 @@ object RowSetUtils {
 val tRows = new java.util.ArrayList[TRow](rowSize)
 while (i < rowSize) {
   val row = rows(i)
-  val tRow = new TRow()
   var j = 0
   val columnSize = row.length
+  val tColumnValues = new java.util.ArrayList[TColumnValue](columnSize)
   while (j < columnSize) {
 val columnValue = toTColumnValue(j, row, schema(j), timeFormatters)
-tRow.addToColVals(columnValue)
+tColumnValues.add(columnValue)
 j += 1
   }
   i += 1
+  val tRow = new TRow(tColumnValues)
   tRows.add(tRow)
 }
 new TRowSet(startRowOffSet, tRows)
@@ -80,11 +81,13 @@ object RowSetUtils {
 val tRowSet = new TRowSet(startRowOffSet, new 
java.util.ArrayList[TRow](rowSize))
 var i = 0
 val columnSize = schema.length
+val tColumns = new java.util.ArrayList[TColumn](columnSize)
 while (i < columnSize) {
   val tColumn = toTColumn(rows, i, schema(i), timeFormatters)
-  tRowSet.addToColumns(tColumn)
+  tColumns.add(tColumn)
   i += 1
 }
+tRowSet.setColumns(tColumns)
 tRowSet
   }
 


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



(spark) 01/01: Preparing development version 3.3.5-SNAPSHOT

2023-12-08 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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

commit 0837e78d9cee986f3f791072df2488947dcf0b9a
Author: Dongjoon Hyun 
AuthorDate: Fri Dec 8 18:28:19 2023 +

Preparing development version 3.3.5-SNAPSHOT
---
 R/pkg/DESCRIPTION  | 2 +-
 assembly/pom.xml   | 2 +-
 common/kvstore/pom.xml | 2 +-
 common/network-common/pom.xml  | 2 +-
 common/network-shuffle/pom.xml | 2 +-
 common/network-yarn/pom.xml| 2 +-
 common/sketch/pom.xml  | 2 +-
 common/tags/pom.xml| 2 +-
 common/unsafe/pom.xml  | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 6 +++---
 examples/pom.xml   | 2 +-
 external/avro/pom.xml  | 2 +-
 external/docker-integration-tests/pom.xml  | 2 +-
 external/kafka-0-10-assembly/pom.xml   | 2 +-
 external/kafka-0-10-sql/pom.xml| 2 +-
 external/kafka-0-10-token-provider/pom.xml | 2 +-
 external/kafka-0-10/pom.xml| 2 +-
 external/kinesis-asl-assembly/pom.xml  | 2 +-
 external/kinesis-asl/pom.xml   | 2 +-
 external/spark-ganglia-lgpl/pom.xml| 2 +-
 graphx/pom.xml | 2 +-
 hadoop-cloud/pom.xml   | 2 +-
 launcher/pom.xml   | 2 +-
 mllib-local/pom.xml| 2 +-
 mllib/pom.xml  | 2 +-
 pom.xml| 2 +-
 python/pyspark/version.py  | 2 +-
 repl/pom.xml   | 2 +-
 resource-managers/kubernetes/core/pom.xml  | 2 +-
 resource-managers/kubernetes/integration-tests/pom.xml | 2 +-
 resource-managers/mesos/pom.xml| 2 +-
 resource-managers/yarn/pom.xml | 2 +-
 sql/catalyst/pom.xml   | 2 +-
 sql/core/pom.xml   | 2 +-
 sql/hive-thriftserver/pom.xml  | 2 +-
 sql/hive/pom.xml   | 2 +-
 streaming/pom.xml  | 2 +-
 tools/pom.xml  | 2 +-
 39 files changed, 41 insertions(+), 41 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index ac01697363e4..6c1d59584cd0 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.3.4
+Version: 3.3.5
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R:
diff --git a/assembly/pom.xml b/assembly/pom.xml
index e3e15e41efb8..11a746d6701d 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4
+3.3.5-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 66fc33f5bb0b..99f36c865fba 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4
+3.3.5-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index fd81fe02bb03..e67d16c8eb9a 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4
+3.3.5-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 9de437c7291a..44432ad56ce2 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4
+3.3.5-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 919717f81a94..6a9377402e77 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4
+3.3.5-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 429544784335..e6baac59e0f2 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4
+3.3.5-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/

(spark) 01/01: Preparing Spark release v3.3.4-rc1

2023-12-08 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to tag v3.3.4-rc1
in repository https://gitbox.apache.org/repos/asf/spark.git

commit 18db204995b32e87a650f2f09f9bcf047ddafa90
Author: Dongjoon Hyun 
AuthorDate: Fri Dec 8 18:28:15 2023 +

Preparing Spark release v3.3.4-rc1
---
 assembly/pom.xml   | 2 +-
 common/kvstore/pom.xml | 2 +-
 common/network-common/pom.xml  | 2 +-
 common/network-shuffle/pom.xml | 2 +-
 common/network-yarn/pom.xml| 2 +-
 common/sketch/pom.xml  | 2 +-
 common/tags/pom.xml| 2 +-
 common/unsafe/pom.xml  | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 2 +-
 examples/pom.xml   | 2 +-
 external/avro/pom.xml  | 2 +-
 external/docker-integration-tests/pom.xml  | 2 +-
 external/kafka-0-10-assembly/pom.xml   | 2 +-
 external/kafka-0-10-sql/pom.xml| 2 +-
 external/kafka-0-10-token-provider/pom.xml | 2 +-
 external/kafka-0-10/pom.xml| 2 +-
 external/kinesis-asl-assembly/pom.xml  | 2 +-
 external/kinesis-asl/pom.xml   | 2 +-
 external/spark-ganglia-lgpl/pom.xml| 2 +-
 graphx/pom.xml | 2 +-
 hadoop-cloud/pom.xml   | 2 +-
 launcher/pom.xml   | 2 +-
 mllib-local/pom.xml| 2 +-
 mllib/pom.xml  | 2 +-
 pom.xml| 2 +-
 python/pyspark/version.py  | 2 +-
 repl/pom.xml   | 2 +-
 resource-managers/kubernetes/core/pom.xml  | 2 +-
 resource-managers/kubernetes/integration-tests/pom.xml | 2 +-
 resource-managers/mesos/pom.xml| 2 +-
 resource-managers/yarn/pom.xml | 2 +-
 sql/catalyst/pom.xml   | 2 +-
 sql/core/pom.xml   | 2 +-
 sql/hive-thriftserver/pom.xml  | 2 +-
 sql/hive/pom.xml   | 2 +-
 streaming/pom.xml  | 2 +-
 tools/pom.xml  | 2 +-
 38 files changed, 38 insertions(+), 38 deletions(-)

diff --git a/assembly/pom.xml b/assembly/pom.xml
index b3d7ee94bb3c..e3e15e41efb8 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4-SNAPSHOT
+3.3.4
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 8a8d44dfd941..66fc33f5bb0b 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4-SNAPSHOT
+3.3.4
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index e76b78305aa7..fd81fe02bb03 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4-SNAPSHOT
+3.3.4
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 92324bf17469..9de437c7291a 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4-SNAPSHOT
+3.3.4
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 6eaf00a3dca6..919717f81a94 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4-SNAPSHOT
+3.3.4
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 3fcff5046da0..429544784335 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4-SNAPSHOT
+3.3.4
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/common/tags/pom.xml
index c04f0eb556e9..6a3243424567 100644
--- a/common/tags/pom.xml
+++ b/common/tags/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.3.4-SNAPSHOT
+3.3.4
 ../../pom.xml
   
 
diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml
index 84a5d404faec..0675a3554c47 100644
--- a/common/unsafe/pom.xml
+++ b/common/unsafe/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark

(spark) branch branch-3.3 updated (6a4488f2f486 -> 0837e78d9cee)

2023-12-08 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


from 6a4488f2f486 [SPARK-45580][SQL][3.3] Handle case where a nested 
subquery becomes an existence join
 add 18db204995b3 Preparing Spark release v3.3.4-rc1
 new 0837e78d9cee Preparing development version 3.3.5-SNAPSHOT

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 R/pkg/DESCRIPTION  | 2 +-
 assembly/pom.xml   | 2 +-
 common/kvstore/pom.xml | 2 +-
 common/network-common/pom.xml  | 2 +-
 common/network-shuffle/pom.xml | 2 +-
 common/network-yarn/pom.xml| 2 +-
 common/sketch/pom.xml  | 2 +-
 common/tags/pom.xml| 2 +-
 common/unsafe/pom.xml  | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 6 +++---
 examples/pom.xml   | 2 +-
 external/avro/pom.xml  | 2 +-
 external/docker-integration-tests/pom.xml  | 2 +-
 external/kafka-0-10-assembly/pom.xml   | 2 +-
 external/kafka-0-10-sql/pom.xml| 2 +-
 external/kafka-0-10-token-provider/pom.xml | 2 +-
 external/kafka-0-10/pom.xml| 2 +-
 external/kinesis-asl-assembly/pom.xml  | 2 +-
 external/kinesis-asl/pom.xml   | 2 +-
 external/spark-ganglia-lgpl/pom.xml| 2 +-
 graphx/pom.xml | 2 +-
 hadoop-cloud/pom.xml   | 2 +-
 launcher/pom.xml   | 2 +-
 mllib-local/pom.xml| 2 +-
 mllib/pom.xml  | 2 +-
 pom.xml| 2 +-
 python/pyspark/version.py  | 2 +-
 repl/pom.xml   | 2 +-
 resource-managers/kubernetes/core/pom.xml  | 2 +-
 resource-managers/kubernetes/integration-tests/pom.xml | 2 +-
 resource-managers/mesos/pom.xml| 2 +-
 resource-managers/yarn/pom.xml | 2 +-
 sql/catalyst/pom.xml   | 2 +-
 sql/core/pom.xml   | 2 +-
 sql/hive-thriftserver/pom.xml  | 2 +-
 sql/hive/pom.xml   | 2 +-
 streaming/pom.xml  | 2 +-
 tools/pom.xml  | 2 +-
 39 files changed, 41 insertions(+), 41 deletions(-)


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



(spark) tag v3.3.4-rc1 created (now 18db204995b3)

2023-12-08 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a change to tag v3.3.4-rc1
in repository https://gitbox.apache.org/repos/asf/spark.git


  at 18db204995b3 (commit)
This tag includes the following new commits:

 new 18db204995b3 Preparing Spark release v3.3.4-rc1

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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



(spark) branch master updated (c06d41859f08 -> 9ffdcc398ed5)

2023-12-07 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 c06d41859f08 [SPARK-46320][CORE] Support `spark.master.rest.host`
 add 9ffdcc398ed5 [SPARK-46321][PS][TESTS] Re-ennable 
`IndexesTests.test_asof` that was skipped due to Pandas bug

No new revisions were added by this update.

Summary of changes:
 python/pyspark/pandas/tests/indexes/test_base.py | 9 -
 1 file changed, 4 insertions(+), 5 deletions(-)


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



(spark) branch master updated (b6b450927ec8 -> c06d41859f08)

2023-12-07 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 b6b450927ec8 [SPARK-46317][PYTHON][CONNECT] Match minor behaviour 
matching in SparkSession with full test coverage
 add c06d41859f08 [SPARK-46320][CORE] Support `spark.master.rest.host`

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala   | 3 ++-
 .../src/main/scala/org/apache/spark/internal/config/package.scala | 6 ++
 docs/spark-standalone.md  | 8 
 3 files changed, 16 insertions(+), 1 deletion(-)


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



(spark) branch master updated: [SPARK-46316][CORE] Enable `buf-lint-action` on `core` module

2023-12-07 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 7e4a63a0588f [SPARK-46316][CORE] Enable `buf-lint-action` on `core` 
module
7e4a63a0588f is described below

commit 7e4a63a0588f1b4b16e76d4d7d1add19cb2f0a82
Author: Dongjoon Hyun 
AuthorDate: Thu Dec 7 19:57:08 2023 -0800

[SPARK-46316][CORE] Enable `buf-lint-action` on `core` module

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

This PR aims to enable `buf-lint-action` on `core` module.

### Why are the changes needed?

To enforce the community guideline.

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

No.

### How was this patch tested?

Pass the CIs.

![Screenshot 2023-12-07 at 7 40 54 
PM](https://github.com/apache/spark/assets/9700541/b23f-d8be-410a-bc61-88f8b477a3b0)

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

No.

Closes #44246 from dongjoon-hyun/SPARK-46316.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .github/workflows/build_and_test.yml |  4 
 core/src/main/protobuf/buf.yaml  | 23 +++
 2 files changed, 27 insertions(+)

diff --git a/.github/workflows/build_and_test.yml 
b/.github/workflows/build_and_test.yml
index 0e1a4a810f8a..e54883552920 100644
--- a/.github/workflows/build_and_test.yml
+++ b/.github/workflows/build_and_test.yml
@@ -583,6 +583,10 @@ jobs:
   uses: bufbuild/buf-setup-action@v1
   with:
 github_token: ${{ secrets.GITHUB_TOKEN }}
+- name: Protocol Buffers Linter
+  uses: bufbuild/buf-lint-action@v1
+  with:
+input: core/src/main/protobuf
 # Change 'branch-3.5' to 'branch-4.0' in master branch after cutting 
branch-4.0 branch.
 - name: Breaking change detection against branch-3.5
   uses: bufbuild/buf-breaking-action@v1
diff --git a/core/src/main/protobuf/buf.yaml b/core/src/main/protobuf/buf.yaml
new file mode 100644
index ..47f69191a5c7
--- /dev/null
+++ b/core/src/main/protobuf/buf.yaml
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+version: v1
+breaking:
+  use:
+- FILE
+lint:
+  use:
+- BASIC


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



(spark) branch master updated: Revert "[SPARK-46316][CORE] Enable `buf-lint-action` on `core` module"

2023-12-07 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 80dc64a573e1 Revert "[SPARK-46316][CORE] Enable `buf-lint-action` on 
`core` module"
80dc64a573e1 is described below

commit 80dc64a573e1c7678f92f8690f09a52329f7d30b
Author: Dongjoon Hyun 
AuthorDate: Thu Dec 7 20:03:01 2023 -0800

Revert "[SPARK-46316][CORE] Enable `buf-lint-action` on `core` module"

This reverts commit dcbae0643ce145df6cd0a7a68af3fdd1a062587b.
---
 .github/workflows/build_and_test.yml |  4 
 core/src/main/protobuf/buf.yaml  | 23 ---
 2 files changed, 27 deletions(-)

diff --git a/.github/workflows/build_and_test.yml 
b/.github/workflows/build_and_test.yml
index e54883552920..0e1a4a810f8a 100644
--- a/.github/workflows/build_and_test.yml
+++ b/.github/workflows/build_and_test.yml
@@ -583,10 +583,6 @@ jobs:
   uses: bufbuild/buf-setup-action@v1
   with:
 github_token: ${{ secrets.GITHUB_TOKEN }}
-- name: Protocol Buffers Linter
-  uses: bufbuild/buf-lint-action@v1
-  with:
-input: core/src/main/protobuf
 # Change 'branch-3.5' to 'branch-4.0' in master branch after cutting 
branch-4.0 branch.
 - name: Breaking change detection against branch-3.5
   uses: bufbuild/buf-breaking-action@v1
diff --git a/core/src/main/protobuf/buf.yaml b/core/src/main/protobuf/buf.yaml
deleted file mode 100644
index 47f69191a5c7..
--- a/core/src/main/protobuf/buf.yaml
+++ /dev/null
@@ -1,23 +0,0 @@
-#
-# 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.
-#
-version: v1
-breaking:
-  use:
-- FILE
-lint:
-  use:
-- BASIC


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



(spark) branch master updated (105eee73cfa0 -> dcbae0643ce1)

2023-12-07 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 105eee73cfa0 [SPARK-46313][CORE] Log `Spark HA` recovery duration
 add dcbae0643ce1 [SPARK-46316][CORE] Enable `buf-lint-action` on `core` 
module

No new revisions were added by this update.

Summary of changes:
 .github/workflows/build_and_test.yml  | 4 
 {connector/connect/common => core}/src/main/protobuf/buf.yaml | 4 +---
 2 files changed, 5 insertions(+), 3 deletions(-)
 copy {connector/connect/common => core}/src/main/protobuf/buf.yaml (94%)


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



(spark) branch master updated (61a3e0587df6 -> 105eee73cfa0)

2023-12-07 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 61a3e0587df6 [SPARK-46312][CORE] Use `lower_camel_case` in 
`store_types.proto`
 add 105eee73cfa0 [SPARK-46313][CORE] Log `Spark HA` recovery duration

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 6 +-
 1 file changed, 5 insertions(+), 1 deletion(-)


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



(spark) branch master updated: [SPARK-46312][CORE] Use `lower_camel_case` in `store_types.proto`

2023-12-07 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 61a3e0587df6 [SPARK-46312][CORE] Use `lower_camel_case` in 
`store_types.proto`
61a3e0587df6 is described below

commit 61a3e0587df6be881cdc115fefb77482fa446b5c
Author: Dongjoon Hyun 
AuthorDate: Thu Dec 7 18:16:21 2023 -0800

[SPARK-46312][CORE] Use `lower_camel_case` in `store_types.proto`

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

This PR aims to use 'lower_camel_case` in `store_types.proto`.

### Why are the changes needed?

According to our guideline, we had better follow 
[FIELD_LOWER_SNAKE_CASE](https://buf.build/docs/lint/rules#field_lower_snake_case)


https://github.com/apache/spark/blob/9585cf6d56e3af37142609668dda1eeda3ec876f/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto#L23

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

No.

### How was this patch tested?

Pass the CIs.

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

No.

Closes #44242 from dongjoon-hyun/SPARK-46312.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../main/protobuf/org/apache/spark/status/protobuf/store_types.proto  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto 
b/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto
index 93365add3a64..386c660b16de 100644
--- a/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto
+++ b/core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto
@@ -164,7 +164,7 @@ message ExecutorStageSummaryWrapper {
 message ExecutorResourceRequest {
   optional string resource_name = 1;
   int64 amount = 2;
-  optional string discoveryScript = 3;
+  optional string discovery_script = 3;
   optional string vendor = 4;
 }
 
@@ -277,7 +277,7 @@ message RDDStorageInfoWrapper {
 }
 
 message ResourceProfileWrapper {
-  ResourceProfileInfo rpInfo = 1;
+  ResourceProfileInfo rp_info = 1;
 }
 
 message CachedQuantile {


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



(spark) branch master updated: [SPARK-46309][PS][TESTS] Remove unused code in `pyspark.pandas.tests.indexes.* `

2023-12-07 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 9585cf6d56e3 [SPARK-46309][PS][TESTS] Remove unused code in 
`pyspark.pandas.tests.indexes.* `
9585cf6d56e3 is described below

commit 9585cf6d56e3af37142609668dda1eeda3ec876f
Author: Ruifeng Zheng 
AuthorDate: Thu Dec 7 15:04:07 2023 -0800

[SPARK-46309][PS][TESTS] Remove unused code in 
`pyspark.pandas.tests.indexes.* `

### What changes were proposed in this pull request?
Remove unused code in `pyspark.pandas.tests.indexes.* `

### Why are the changes needed?
clean up the code

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

### How was this patch tested?
ci

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

Closes #44239 from zhengruifeng/ps_index_cleanup.

Authored-by: Ruifeng Zheng 
Signed-off-by: Dongjoon Hyun 
---
 .../pyspark/pandas/tests/data_type_ops/test_string_ops.py  |  4 
 python/pyspark/pandas/tests/indexes/test_align.py  | 14 --
 python/pyspark/pandas/tests/indexes/test_base_slow.py  |  7 ---
 python/pyspark/pandas/tests/indexes/test_reindex.py| 14 --
 python/pyspark/pandas/tests/indexes/test_rename.py | 14 --
 5 files changed, 53 deletions(-)

diff --git a/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py 
b/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py
index 2870aed8e75e..340153b06335 100644
--- a/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py
+++ b/python/pyspark/pandas/tests/data_type_ops/test_string_ops.py
@@ -35,10 +35,6 @@ class StringOpsTestsMixin:
 def bool_pdf(self):
 return pd.DataFrame({"this": ["x", "y", "z"], "that": ["z", "y", "x"]})
 
-@property
-def bool_psdf(self):
-return ps.from_pandas(self.bool_pdf)
-
 @property
 def bool_non_numeric_pdf(self):
 return pd.concat([self.bool_pdf, self.non_numeric_pdf], axis=1)
diff --git a/python/pyspark/pandas/tests/indexes/test_align.py 
b/python/pyspark/pandas/tests/indexes/test_align.py
index 56fde9b4f28b..73e10d441078 100644
--- a/python/pyspark/pandas/tests/indexes/test_align.py
+++ b/python/pyspark/pandas/tests/indexes/test_align.py
@@ -16,7 +16,6 @@
 #
 import unittest
 
-import numpy as np
 import pandas as pd
 
 from pyspark import pandas as ps
@@ -25,19 +24,6 @@ from pyspark.testing.sqlutils import SQLTestUtils
 
 
 class FrameAlignMixin:
-@property
-def pdf(self):
-return pd.DataFrame(
-{"a": [1, 2, 3, 4, 5, 6, 7, 8, 9], "b": [4, 5, 6, 3, 2, 1, 0, 0, 
0]},
-index=np.random.rand(9),
-)
-
-@property
-def df_pair(self):
-pdf = self.pdf
-psdf = ps.from_pandas(pdf)
-return pdf, psdf
-
 def test_align(self):
 pdf1 = pd.DataFrame({"a": [1, 2, 3], "b": ["a", "b", "c"]}, index=[10, 
20, 30])
 psdf1 = ps.from_pandas(pdf1)
diff --git a/python/pyspark/pandas/tests/indexes/test_base_slow.py 
b/python/pyspark/pandas/tests/indexes/test_base_slow.py
index c890f5004b43..eb417fe47ef8 100644
--- a/python/pyspark/pandas/tests/indexes/test_base_slow.py
+++ b/python/pyspark/pandas/tests/indexes/test_base_slow.py
@@ -24,13 +24,6 @@ from pyspark.testing.pandasutils import ComparisonTestBase, 
TestUtils
 
 
 class IndexesSlowTestsMixin:
-@property
-def pdf(self):
-return pd.DataFrame(
-{"a": [1, 2, 3, 4, 5, 6, 7, 8, 9], "b": [4, 5, 6, 3, 2, 1, 0, 0, 
0]},
-index=[0, 1, 3, 5, 6, 8, 9, 9, 9],
-)
-
 def test_append(self):
 # Index
 pidx = pd.Index(range(1))
diff --git a/python/pyspark/pandas/tests/indexes/test_reindex.py 
b/python/pyspark/pandas/tests/indexes/test_reindex.py
index 1d544ea221bf..1229a613846b 100644
--- a/python/pyspark/pandas/tests/indexes/test_reindex.py
+++ b/python/pyspark/pandas/tests/indexes/test_reindex.py
@@ -16,7 +16,6 @@
 #
 import unittest
 
-import numpy as np
 import pandas as pd
 
 from pyspark import pandas as ps
@@ -25,19 +24,6 @@ from pyspark.testing.sqlutils import SQLTestUtils
 
 
 class FrameReindexMixin:
-@property
-def pdf(self):
-return pd.DataFrame(
-{"a": [1, 2, 3, 4, 5, 6, 7, 8, 9], "b": [4, 5, 6, 3, 2, 1, 0, 0, 
0]},
-index=np.random.rand(9),
-)
-
-@property
-def df_pair(self):
-pdf = self.pdf
-psdf = ps.from_pandas(pdf)
-return pdf, psdf
-
 def test_reindex(self):
 index = pd.Index(["A", "B", "C", "D", &q

(spark) branch master updated (027aeb1764a8 -> 82e67461511e)

2023-12-07 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 027aeb1764a8 [SPARK-46277][PYTHON] Validate startup urls with the 
config being set
 add 82e67461511e [SPARK-46311][CORE] Log the final state of drivers during 
`Master.removeDriver`

No new revisions were added by this update.

Summary of changes:
 core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)


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



(spark) branch master updated: [SPARK-46293][CONNECT][PYTHON] Use `protobuf` transitive dependency

2023-12-07 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 e359318c4493 [SPARK-46293][CONNECT][PYTHON] Use `protobuf` transitive 
dependency
e359318c4493 is described below

commit e359318c4493e16a7546d70c9340ffc5015aacff
Author: Haejoon Lee 
AuthorDate: Thu Dec 7 10:28:27 2023 -0800

[SPARK-46293][CONNECT][PYTHON] Use `protobuf` transitive dependency

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

This PR proposes to remove `protobuf` from required package.

### Why are the changes needed?

`protobuf` is automatically installed when installing `grpcio` and 
`grpcio-status`, so we don't need to specify the specific version explicitly.

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

No API changes.

### How was this patch tested?

The existing CI should pass

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

No.

Closes #44221 from itholic/protobuf_docs.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 dev/requirements.txt | 1 -
 1 file changed, 1 deletion(-)

diff --git a/dev/requirements.txt b/dev/requirements.txt
index 0f1f1aee5b63..51facfeb5088 100644
--- a/dev/requirements.txt
+++ b/dev/requirements.txt
@@ -53,7 +53,6 @@ py
 # Spark Connect (required)
 grpcio>=1.59.3
 grpcio-status>=1.59.3
-protobuf==4.25.1
 googleapis-common-protos>=1.56.4
 
 # Spark Connect python proto generation plugin (optional)


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



(spark) branch master updated (0692856bb124 -> 8132e1700c81)

2023-12-07 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 0692856bb124 [SPARK-46307][PS][TESTS] Enable `fill_value` tests for 
`GroupByTests.test_shift`
 add 8132e1700c81 [SPARK-46261][CONNECT] `DataFrame.withColumnsRenamed` 
should keep the dict/map ordering

No new revisions were added by this update.

Summary of changes:
 .../main/scala/org/apache/spark/sql/Dataset.scala  |   8 ++-
 .../main/protobuf/spark/connect/relations.proto|  14 +++-
 .../queries/withColumnRenamed_java_map.json|  11 +--
 .../queries/withColumnRenamed_java_map.proto.bin   | Bin 72 -> 72 bytes
 .../queries/withColumnRenamed_scala_map.json   |  11 +--
 .../queries/withColumnRenamed_scala_map.proto.bin  | Bin 72 -> 72 bytes
 .../queries/withColumnRenamed_single.json  |   7 +-
 .../queries/withColumnRenamed_single.proto.bin | Bin 60 -> 60 bytes
 .../sql/connect/planner/SparkConnectPlanner.scala  |  19 +++--
 python/pyspark/sql/connect/plan.py |   8 ++-
 python/pyspark/sql/connect/proto/relations_pb2.py  |  80 +++--
 python/pyspark/sql/connect/proto/relations_pb2.pyi |  34 -
 .../sql/tests/connect/test_parity_dataframe.py |   5 --
 .../main/scala/org/apache/spark/sql/Dataset.scala  |   2 +-
 14 files changed, 133 insertions(+), 66 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-46301][CORE] Support `spark.worker.(initial|max)RegistrationRetries`

2023-12-07 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 b92d64d6ef0c [SPARK-46301][CORE] Support 
`spark.worker.(initial|max)RegistrationRetries`
b92d64d6ef0c is described below

commit b92d64d6ef0c99b6b444f41ebdfe95f3260312aa
Author: Dongjoon Hyun 
AuthorDate: Thu Dec 7 01:01:56 2023 -0800

[SPARK-46301][CORE] Support `spark.worker.(initial|max)RegistrationRetries`

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

This PR aims to support `spark.worker.(initial|max)RegistrationRetries` to 
parameterize the hard-coded magic numbers.
```scala
- private val INITIAL_REGISTRATION_RETRIES = 6
- private val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10
+ private val INITIAL_REGISTRATION_RETRIES = 
conf.get(WORKER_INITIAL_REGISTRATION_RETRIES)
+ private val TOTAL_REGISTRATION_RETRIES = 
conf.get(WORKER_MAX_REGISTRATION_RETRIES)
```

### Why are the changes needed?

To allow users to control these.

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

No. The default values are consistent with the existing behavior.

### How was this patch tested?

Pass the CIs.

![Screenshot 2023-12-06 at 8 58 05 
PM](https://github.com/apache/spark/assets/9700541/985ff3f7-d8c9-4803-a207-a6c16388e4d0)

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

No.

Closes #44229 from dongjoon-hyun/SPARK-46301.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../scala/org/apache/spark/deploy/worker/Worker.scala  | 14 ++
 .../org/apache/spark/internal/config/Worker.scala  | 17 +
 docs/spark-standalone.md   | 18 ++
 3 files changed, 45 insertions(+), 4 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala 
b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index eae12648b95a..1422a1484f8d 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -96,12 +96,18 @@ private[deploy] class Worker(
   private val HEARTBEAT_MILLIS = conf.get(WORKER_TIMEOUT) * 1000 / 4
 
   // Model retries to connect to the master, after Hadoop's model.
-  // The first six attempts to reconnect are in shorter intervals (between 5 
and 15 seconds)
-  // Afterwards, the next 10 attempts are between 30 and 90 seconds.
+  // The total number of retries are less than or equal to 
WORKER_MAX_REGISTRATION_RETRIES.
+  // Within the upper limit, WORKER_MAX_REGISTRATION_RETRIES,
+  // the first WORKER_INITIAL_REGISTRATION_RETRIES attempts to reconnect are 
in shorter intervals
+  // (between 5 and 15 seconds). Afterwards, the next attempts are between 30 
and 90 seconds while
   // A bit of randomness is introduced so that not all of the workers attempt 
to reconnect at
   // the same time.
-  private val INITIAL_REGISTRATION_RETRIES = 6
-  private val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10
+  private val INITIAL_REGISTRATION_RETRIES = 
conf.get(WORKER_INITIAL_REGISTRATION_RETRIES)
+  private val TOTAL_REGISTRATION_RETRIES = 
conf.get(WORKER_MAX_REGISTRATION_RETRIES)
+  if (INITIAL_REGISTRATION_RETRIES > TOTAL_REGISTRATION_RETRIES) {
+logInfo(s"${WORKER_INITIAL_REGISTRATION_RETRIES.key} 
($INITIAL_REGISTRATION_RETRIES) is " +
+  s"capped by ${WORKER_MAX_REGISTRATION_RETRIES.key} 
($TOTAL_REGISTRATION_RETRIES)")
+  }
   private val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500
   private val REGISTRATION_RETRY_FUZZ_MULTIPLIER = {
 val randomNumberGenerator = new 
Random(UUID.randomUUID.getMostSignificantBits)
diff --git a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala 
b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala
index f160470edd8f..c53e181df002 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala
@@ -37,6 +37,23 @@ private[spark] object Worker {
 .longConf
 .createWithDefault(60)
 
+  val WORKER_INITIAL_REGISTRATION_RETRIES = 
ConfigBuilder("spark.worker.initialRegistrationRetries")
+.version("4.0.0")
+.internal()
+.doc("The number of retries to reconnect in short intervals (between 5 and 
15 seconds).")
+.intConf
+.checkValue(_ > 0, "The number of initial registration retries should be 
positive")
+.createWithDefault(6)
+
+  val WORKER_MAX_REGISTRATION_RETRIES = 
ConfigBuilder("spark.worker.maxRegistrationRetries")
+.version("4.0.0")
+.internal()
+.doc(&

(spark) branch branch-3.3 updated: [SPARK-45580][SQL][3.3] Handle case where a nested subquery becomes an existence join

2023-12-06 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 6a4488f2f48 [SPARK-45580][SQL][3.3] Handle case where a nested 
subquery becomes an existence join
6a4488f2f48 is described below

commit 6a4488f2f4861df41025480cceda643e9e74484e
Author: Bruce Robbins 
AuthorDate: Wed Dec 6 19:24:13 2023 -0800

[SPARK-45580][SQL][3.3] Handle case where a nested subquery becomes an 
existence join

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

This is a back-port of https://github.com/apache/spark/pull/44193.

In `RewritePredicateSubquery`, prune existence flags from the final join 
when `rewriteExistentialExpr` returns an existence join. This change prunes the 
flags (attributes with the name "exists") by adding a `Project` node.

For example:
```
Join LeftSemi, ((a#13 = c1#15) OR exists#19)
:- Join ExistenceJoin(exists#19), (a#13 = col1#17)
:  :- LocalRelation [a#13]
:  +- LocalRelation [col1#17]
+- LocalRelation [c1#15]
```
becomes
```
Project [a#13]
+- Join LeftSemi, ((a#13 = c1#15) OR exists#19)
   :- Join ExistenceJoin(exists#19), (a#13 = col1#17)
   :  :- LocalRelation [a#13]
   :  +- LocalRelation [col1#17]
   +- LocalRelation [c1#15]
```
This change always adds the `Project` node, whether 
`rewriteExistentialExpr` returns an existence join or not. In the case when 
`rewriteExistentialExpr` does not return an existence join, 
`RemoveNoopOperators` will remove the unneeded `Project` node.

### Why are the changes needed?

This query returns an extraneous boolean column when run in spark-sql:
```
create or replace temp view t1(a) as values (1), (2), (3), (7);
create or replace temp view t2(c1) as values (1), (2), (3);
create or replace temp view t3(col1) as values (3), (9);

select *
from t1
where exists (
  select c1
  from t2
  where a = c1
  or a in (select col1 from t3)
);

1   false
2   false
3   true
```
(Note: the above query will not have the extraneous boolean column when run 
from the Dataset API. That is because the Dataset API truncates the rows based 
on the schema of the analyzed plan. The bug occurs during optimization).

This query fails when run in either spark-sql or using the Dataset API:
```
select (
  select *
  from t1
  where exists (
select c1
from t2
where a = c1
or a in (select col1 from t3)
  )
  limit 1
)
from range(1);

java.lang.AssertionError: assertion failed: Expects 1 field, but got 2; 
something went wrong in analysis
```

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

No, except for the removal of the extraneous boolean flag and the fix to 
the error condition.

### How was this patch tested?

New unit test.

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

No.

Closes #44223 from bersprockets/schema_change_br33.

Authored-by: Bruce Robbins 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/sql/catalyst/optimizer/subquery.scala|  9 +++--
 .../scala/org/apache/spark/sql/SubquerySuite.scala | 46 ++
 2 files changed, 52 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
index 7ef5ef55fab..ff198c798b9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
@@ -113,16 +113,19 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] 
with PredicateHelper {
   withSubquery.foldLeft(newFilter) {
 case (p, Exists(sub, _, _, conditions)) =>
   val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftSemi, joinCond)
+  val join = buildJoin(outerPlan, sub, LeftSemi, joinCond)
+  Project(p.output, join)
 case (p, Not(Exists(sub, _, _, conditions))) =>
   val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftAnti, joinCond)
+  val join = buildJoin(outerPlan, sub, LeftAnti, joinCond)
+  Project(p.output, join)
 case (p, InSubquery(values, ListQuery(sub, _, _, _, conditions))) =>
   // Deduplicate conflicting attributes if any.
   val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values))
   val inConditions = values.zip(newSub.output).map(EqualTo.tupled)
 

(spark) branch branch-3.4 updated: [SPARK-45580][SQL][3.4] Handle case where a nested subquery becomes an existence join

2023-12-06 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 8e40ec6fa52 [SPARK-45580][SQL][3.4] Handle case where a nested 
subquery becomes an existence join
8e40ec6fa52 is described below

commit 8e40ec6fa525420c1da5ce3b8846ef9f540b9d49
Author: Bruce Robbins 
AuthorDate: Wed Dec 6 19:23:19 2023 -0800

[SPARK-45580][SQL][3.4] Handle case where a nested subquery becomes an 
existence join

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

This is a back-port of https://github.com/apache/spark/pull/44193.

In `RewritePredicateSubquery`, prune existence flags from the final join 
when `rewriteExistentialExpr` returns an existence join. This change prunes the 
flags (attributes with the name "exists") by adding a `Project` node.

For example:
```
Join LeftSemi, ((a#13 = c1#15) OR exists#19)
:- Join ExistenceJoin(exists#19), (a#13 = col1#17)
:  :- LocalRelation [a#13]
:  +- LocalRelation [col1#17]
+- LocalRelation [c1#15]
```
becomes
```
Project [a#13]
+- Join LeftSemi, ((a#13 = c1#15) OR exists#19)
   :- Join ExistenceJoin(exists#19), (a#13 = col1#17)
   :  :- LocalRelation [a#13]
   :  +- LocalRelation [col1#17]
   +- LocalRelation [c1#15]
```
This change always adds the `Project` node, whether 
`rewriteExistentialExpr` returns an existence join or not. In the case when 
`rewriteExistentialExpr` does not return an existence join, 
`RemoveNoopOperators` will remove the unneeded `Project` node.

### Why are the changes needed?

This query returns an extraneous boolean column when run in spark-sql:
```
create or replace temp view t1(a) as values (1), (2), (3), (7);
create or replace temp view t2(c1) as values (1), (2), (3);
create or replace temp view t3(col1) as values (3), (9);

select *
from t1
where exists (
  select c1
  from t2
  where a = c1
  or a in (select col1 from t3)
);

1   false
2   false
3   true
```
(Note: the above query will not have the extraneous boolean column when run 
from the Dataset API. That is because the Dataset API truncates the rows based 
on the schema of the analyzed plan. The bug occurs during optimization).

This query fails when run in either spark-sql or using the Dataset API:
```
select (
  select *
  from t1
  where exists (
select c1
from t2
where a = c1
or a in (select col1 from t3)
  )
  limit 1
)
from range(1);

java.lang.AssertionError: assertion failed: Expects 1 field, but got 2; 
something went wrong in analysis
```

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

No, except for the removal of the extraneous boolean flag and the fix to 
the error condition.

### How was this patch tested?

New unit test.

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

No.

Closes #44219 from bersprockets/schema_change_br34.

Authored-by: Bruce Robbins 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/sql/catalyst/optimizer/subquery.scala|  9 +++--
 .../scala/org/apache/spark/sql/SubquerySuite.scala | 46 ++
 2 files changed, 52 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
index 1d2f5602630..861f2f2fabf 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
@@ -118,16 +118,19 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] 
with PredicateHelper {
   withSubquery.foldLeft(newFilter) {
 case (p, Exists(sub, _, _, conditions, subHint)) =>
   val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftSemi, joinCond, subHint)
+  val join = buildJoin(outerPlan, sub, LeftSemi, joinCond, subHint)
+  Project(p.output, join)
 case (p, Not(Exists(sub, _, _, conditions, subHint))) =>
   val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftAnti, joinCond, subHint)
+  val join = buildJoin(outerPlan, sub, LeftAnti, joinCond, subHint)
+  Project(p.output, join)
 case (p, InSubquery(values, ListQuery(sub, _, _, _, conditions, 
subHint))) =>
   // Deduplicate conflicting attributes if any.
   val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values))
   val inCond

(spark) branch master updated: [SPARK-46299][DOCS] Make `spark.deploy.recovery*` docs up-to-date

2023-12-06 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 43ca0b929ab [SPARK-46299][DOCS] Make `spark.deploy.recovery*` docs 
up-to-date
43ca0b929ab is described below

commit 43ca0b929ab3c2f10d1879e5df622195564f8885
Author: Dongjoon Hyun 
AuthorDate: Wed Dec 6 19:19:41 2023 -0800

[SPARK-46299][DOCS] Make `spark.deploy.recovery*` docs up-to-date

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

This PR aims to update `Spark Standalone` cluster recovery configurations.

### Why are the changes needed?

We need to document
- #44173
- #44129
- #44113

![Screenshot 2023-12-06 at 7 15 24 
PM](https://github.com/apache/spark/assets/9700541/04f0be6f-cdfb-4d87-b1b5-c4bf131f460a)

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

No.

### How was this patch tested?

Manual review.

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

No.

Closes #44227 from dongjoon-hyun/SPARK-46299.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 docs/spark-standalone.md | 26 +++---
 1 file changed, 23 insertions(+), 3 deletions(-)

diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 7a89c8124bd..25d2fba47ce 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -735,18 +735,38 @@ In order to enable this recovery mode, you can set 
SPARK_DAEMON_JAVA_OPTS in spa
   
 spark.deploy.recoveryMode
 NONE
-The recovery mode setting to recover submitted Spark jobs with cluster 
mode when it failed and relaunches.
-  Set to FILESYSTEM to enable single-node recovery mode, ZOOKEEPER to use 
Zookeeper-based recovery mode, and
+The recovery mode setting to recover submitted Spark jobs with cluster 
mode when it failed and relaunches. Set to
+  FILESYSTEM to enable file-system-based single-node recovery mode,
+  ROCKSDB to enable RocksDB-based single-node recovery mode,
+  ZOOKEEPER to use Zookeeper-based recovery mode, and
   CUSTOM to provide a customer provider class via additional 
`spark.deploy.recoveryMode.factory` configuration.
+  NONE is the default value which disables this recovery mode.
 
 0.8.1
   
   
 spark.deploy.recoveryDirectory
 ""
-The directory in which Spark will store recovery state, accessible 
from the Master's perspective.
+The directory in which Spark will store recovery state, accessible 
from the Master's perspective.
+  Note that the directory should be clearly manualy if 
spark.deploy.recoveryMode,
+  spark.deploy.recoverySerializer, or 
spark.deploy.recoveryCompressionCodec is changed.
+
 0.8.1
   
+  
+spark.deploy.recoverySerializer
+JAVA
+A serializer for writing/reading objects to/from persistence engines; 
JAVA (default) or KRYO.
+  Java serializer has been the default mode since Spark 0.8.1.
+  Kryo serializer is a new fast and compact mode from Spark 4.0.0.
+4.0.0
+  
+  
+spark.deploy.recoveryCompressionCodec
+(none)
+A compression codec for persistence engines. none (default), lz4, lzf, 
snappy, and zstd. Currently, only FILESYSTEM mode supports this 
configuration.
+4.0.0
+  
   
 spark.deploy.recoveryMode.factory
 ""


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



(spark) branch master updated: [SPARK-46292][CORE][UI] Show a summary of workers in MasterPage

2023-12-06 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 e2441c41de4 [SPARK-46292][CORE][UI] Show a summary of workers in 
MasterPage
e2441c41de4 is described below

commit e2441c41de476b09542db60836d7d853d47f6158
Author: Dongjoon Hyun 
AuthorDate: Wed Dec 6 17:49:37 2023 -0800

[SPARK-46292][CORE][UI] Show a summary of workers in MasterPage

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

This PR aims to show a summary of workers in MasterPage.

### Why are the changes needed?

Although `Alive Workers` is a useful information, it's insufficient to 
analyze the whole cluster status because we don't know how many workers are in 
other status. Especially, this is useful during the recovery process of Spark 
Master HA setting.

In short, this helps the users identify the issues intuitively.

```
- Alive Workers: 1
+ Workers: 1 Alive, 1 Dead, 0 Decommissioned, 0 Unknown
```

Here is a screenshot.

![Screenshot 2023-12-06 at 3 13 43 
PM](https://github.com/apache/spark/assets/9700541/f078b6ae-ab22-4721-8c67-661121bb9807)

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

No.

### How was this patch tested?

Manual test.

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

No.

Closes #44218 from dongjoon-hyun/SPARK-46292.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala   | 6 +-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git 
a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index b2f35984d37..f25e3495d79 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -144,7 +144,11 @@ private[ui] class MasterPage(parent: MasterWebUI) extends 
WebUIPage("") {
   
 }.getOrElse { Seq.empty }
   }
-  Alive Workers: {aliveWorkers.length}
+  Workers: {aliveWorkers.length} Alive,
+{workers.count(_.state == WorkerState.DEAD)} Dead,
+{workers.count(_.state == WorkerState.DECOMMISSIONED)} 
Decommissioned,
+{workers.count(_.state == WorkerState.UNKNOWN)} Unknown
+  
   Cores in use: 
{aliveWorkers.map(_.cores).sum} Total,
 {aliveWorkers.map(_.coresUsed).sum} Used
   Memory in use:


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



(spark) branch branch-3.5 updated: [SPARK-45580][SQL][3.5] Handle case where a nested subquery becomes an existence join

2023-12-06 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new dbb61981b80 [SPARK-45580][SQL][3.5] Handle case where a nested 
subquery becomes an existence join
dbb61981b80 is described below

commit dbb61981b804dbc03cf140c7c76653348e2ac740
Author: Bruce Robbins 
AuthorDate: Wed Dec 6 15:24:48 2023 -0800

[SPARK-45580][SQL][3.5] Handle case where a nested subquery becomes an 
existence join

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

This is a back-port of #44193.

In `RewritePredicateSubquery`, prune existence flags from the final join 
when `rewriteExistentialExpr` returns an existence join. This change prunes the 
flags (attributes with the name "exists") by adding a `Project` node.

For example:
```
Join LeftSemi, ((a#13 = c1#15) OR exists#19)
:- Join ExistenceJoin(exists#19), (a#13 = col1#17)
:  :- LocalRelation [a#13]
:  +- LocalRelation [col1#17]
+- LocalRelation [c1#15]
```
becomes
```
Project [a#13]
+- Join LeftSemi, ((a#13 = c1#15) OR exists#19)
   :- Join ExistenceJoin(exists#19), (a#13 = col1#17)
   :  :- LocalRelation [a#13]
   :  +- LocalRelation [col1#17]
   +- LocalRelation [c1#15]
```
This change always adds the `Project` node, whether 
`rewriteExistentialExpr` returns an existence join or not. In the case when 
`rewriteExistentialExpr` does not return an existence join, 
`RemoveNoopOperators` will remove the unneeded `Project` node.

### Why are the changes needed?

This query returns an extraneous boolean column when run in spark-sql:
```
create or replace temp view t1(a) as values (1), (2), (3), (7);
create or replace temp view t2(c1) as values (1), (2), (3);
create or replace temp view t3(col1) as values (3), (9);

select *
from t1
where exists (
  select c1
  from t2
  where a = c1
  or a in (select col1 from t3)
);

1   false
2   false
3   true
```
(Note: the above query will not have the extraneous boolean column when run 
from the Dataset API. That is because the Dataset API truncates the rows based 
on the schema of the analyzed plan. The bug occurs during optimization).

This query fails when run in either spark-sql or using the Dataset API:
```
select (
  select *
  from t1
  where exists (
select c1
from t2
where a = c1
or a in (select col1 from t3)
  )
  limit 1
)
from range(1);

java.lang.AssertionError: assertion failed: Expects 1 field, but got 2; 
something went wrong in analysis
```

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

No, except for the removal of the extraneous boolean flag and the fix to 
the error condition.

### How was this patch tested?

New unit test.

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

No.

Closes #44215 from bersprockets/schema_change_br35.

Authored-by: Bruce Robbins 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/sql/catalyst/optimizer/subquery.scala|  9 +++--
 .../scala/org/apache/spark/sql/SubquerySuite.scala | 46 ++
 2 files changed, 52 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
index 91cd838ad61..ee200531578 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
@@ -118,16 +118,19 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] 
with PredicateHelper {
   withSubquery.foldLeft(newFilter) {
 case (p, Exists(sub, _, _, conditions, subHint)) =>
   val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftSemi, joinCond, subHint)
+  val join = buildJoin(outerPlan, sub, LeftSemi, joinCond, subHint)
+  Project(p.output, join)
 case (p, Not(Exists(sub, _, _, conditions, subHint))) =>
   val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, sub, LeftAnti, joinCond, subHint)
+  val join = buildJoin(outerPlan, sub, LeftAnti, joinCond, subHint)
+  Project(p.output, join)
 case (p, InSubquery(values, ListQuery(sub, _, _, _, conditions, 
subHint))) =>
   // Deduplicate conflicting attributes if any.
   val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values))
   val inConditions = values.zip(newSub.output)

(spark) branch master updated: [SPARK-46230][PYTHON] Migrate `RetriesExceeded` into PySpark error

2023-12-06 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 31a48381f51 [SPARK-46230][PYTHON] Migrate `RetriesExceeded` into 
PySpark error
31a48381f51 is described below

commit 31a48381f5139a51045a10df344df3ce7ad1adb7
Author: Haejoon Lee 
AuthorDate: Wed Dec 6 11:00:43 2023 -0800

[SPARK-46230][PYTHON] Migrate `RetriesExceeded` into PySpark error

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

This PR proposes to migrate `RetriesExceeded` into PySpark error.

### Why are the changes needed?

All errors defined from PySpark should be inherits `PySparkException` to 
keep the consistency of error messages generated from PySpark.

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

No, it's internal refactoring for better error handling.

### How was this patch tested?

The existing CI should pass.

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

No.

Closes #44147 from itholic/retires_exception.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 python/docs/source/reference/pyspark.errors.rst|  1 +
 python/pyspark/errors/__init__.py  |  2 ++
 python/pyspark/errors/error_classes.py |  5 +
 python/pyspark/errors/exceptions/base.py   |  7 +++
 python/pyspark/sql/connect/client/retries.py   | 11 ++-
 python/pyspark/sql/tests/connect/client/test_client.py |  2 +-
 python/pyspark/sql/tests/connect/test_connect_basic.py |  3 ++-
 7 files changed, 20 insertions(+), 11 deletions(-)

diff --git a/python/docs/source/reference/pyspark.errors.rst 
b/python/docs/source/reference/pyspark.errors.rst
index a4997506b41..270a8a8c716 100644
--- a/python/docs/source/reference/pyspark.errors.rst
+++ b/python/docs/source/reference/pyspark.errors.rst
@@ -48,6 +48,7 @@ Classes
 PySparkIndexError
 PythonException
 QueryExecutionException
+RetriesExceeded
 SessionNotSameException
 SparkRuntimeException
 SparkUpgradeException
diff --git a/python/pyspark/errors/__init__.py 
b/python/pyspark/errors/__init__.py
index 07033d21643..a4f64e85f87 100644
--- a/python/pyspark/errors/__init__.py
+++ b/python/pyspark/errors/__init__.py
@@ -46,6 +46,7 @@ from pyspark.errors.exceptions.base import (  # noqa: F401
 PySparkAssertionError,
 PySparkNotImplementedError,
 PySparkPicklingError,
+RetriesExceeded,
 PySparkKeyError,
 )
 
@@ -78,5 +79,6 @@ __all__ = [
 "PySparkAssertionError",
 "PySparkNotImplementedError",
 "PySparkPicklingError",
+"RetriesExceeded",
 "PySparkKeyError",
 ]
diff --git a/python/pyspark/errors/error_classes.py 
b/python/pyspark/errors/error_classes.py
index c93ffa94149..965fd04a913 100644
--- a/python/pyspark/errors/error_classes.py
+++ b/python/pyspark/errors/error_classes.py
@@ -813,6 +813,11 @@ ERROR_CLASSES_JSON = """
   "Columns do not match in their data type: ."
 ]
   },
+  "RETRIES_EXCEEDED" : {
+"message" : [
+  "The maximum number of retries has been exceeded."
+]
+  },
   "SCHEMA_MISMATCH_FOR_PANDAS_UDF" : {
 "message" : [
   "Result vector from pandas_udf was not the required length: expected 
, got ."
diff --git a/python/pyspark/errors/exceptions/base.py 
b/python/pyspark/errors/exceptions/base.py
index b7d8ed88ec0..b60800da3ff 100644
--- a/python/pyspark/errors/exceptions/base.py
+++ b/python/pyspark/errors/exceptions/base.py
@@ -260,6 +260,13 @@ class PySparkPicklingError(PySparkException, 
PicklingError):
 """
 
 
+class RetriesExceeded(PySparkException):
+"""
+Represents an exception which is considered retriable, but retry limits
+were exceeded
+"""
+
+
 class PySparkKeyError(PySparkException, KeyError):
 """
 Wrapper class for KeyError to support error classes.
diff --git a/python/pyspark/sql/connect/client/retries.py 
b/python/pyspark/sql/connect/client/retries.py
index 88fc3fe1ffd..44e5e1834a2 100644
--- a/python/pyspark/sql/connect/client/retries.py
+++ b/python/pyspark/sql/connect/client/retries.py
@@ -22,7 +22,7 @@ import typing
 from typing import Optional, Callable, Generator, List, Type
 from types import TracebackType
 from pyspark.sql.connect.client.logging import logger
-from pyspark.errors import PySparkRuntimeError
+from pyspark.errors import PySparkRuntimeError, RetriesExceeded
 
 """
 This module contains retry system. The system is designed to be
@@ -233,7 +233,7 @@ class Retrying:
 
 # Exceeded retries
 logger.debug(f"Given 

(spark) branch master updated: [SPARK-46270][SQL][CORE][SS] Use java16 instanceof expressions to replace the java8 instanceof statement

2023-12-06 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 231d89f89ed [SPARK-46270][SQL][CORE][SS] Use java16 instanceof 
expressions to replace the java8 instanceof statement
231d89f89ed is described below

commit 231d89f89ede2cac6cad596f2a3b36673ad0b2f3
Author: Jiaan Geng 
AuthorDate: Wed Dec 6 10:59:39 2023 -0800

[SPARK-46270][SQL][CORE][SS] Use java16 instanceof expressions to replace 
the java8 instanceof statement

### What changes were proposed in this pull request?
This PR uses java14 `instanceof` to replace the java8 `instanceof`.
For example:
```
if (obj instanceof String) {
String s = (String) obj;// grr...
...
}
```
We can change it to
```
if (obj instanceof String s) {
// Let pattern matching do the work!
...
}
```

### Why are the changes needed?
Using [`[JEP 394: Pattern Matching for instanceof]` 
](https://openjdk.org/jeps/394)can bring the following benefits:

1. **More concise syntax**: Pattern matching allows the desired "shape" of 
an object to be expressed concisely (the pattern), and for various statements 
and expressions to test that "shape" against their input (the matching).

2. **Safer**: The motto is: "A pattern variable is in scope where it has 
definitely matched". This allows for the safe reuse of pattern variables and is 
both intuitive and familiar, since Java developers are already used to flow 
sensitive analyses.

3. **Avoid explicit casts**: The use of pattern matching in instanceof 
should significantly reduce the overall number of explicit casts in Java 
programs. Type test patterns are particularly useful when writing equality 
methods.

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

### How was this patch tested?
GA

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

Closes #44187 from beliefer/SPARK-46270.

Authored-by: Jiaan Geng 
Signed-off-by: Dongjoon Hyun 
---
 .../apache/spark/util/kvstore/ArrayWrappers.java   | 28 
 .../spark/util/kvstore/KVStoreSerializer.java  |  4 +-
 .../apache/spark/util/kvstore/LevelDBTypeInfo.java | 12 ++--
 .../apache/spark/util/kvstore/RocksDBTypeInfo.java | 12 ++--
 .../spark/network/client/StreamInterceptor.java|  4 +-
 .../network/client/TransportResponseHandler.java   | 21 ++
 .../protocol/EncryptedMessageWithHeader.java   | 14 ++--
 .../spark/network/protocol/MessageWithHeader.java  |  8 +--
 .../spark/network/protocol/SslMessageEncoder.java  |  7 +-
 .../spark/network/sasl/SaslClientBootstrap.java|  4 +-
 .../network/server/TransportChannelHandler.java|  8 +--
 .../network/server/TransportRequestHandler.java| 28 
 .../network/ssl/ReloadingX509TrustManager.java |  4 +-
 .../org/apache/spark/network/ssl/SSLFactory.java   |  4 +-
 .../org/apache/spark/network/util/NettyLogger.java | 13 ++--
 .../apache/spark/network/TestManagedBuffer.java|  4 +-
 .../spark/network/crypto/AuthIntegrationSuite.java |  4 +-
 .../apache/spark/network/shuffle/ErrorHandler.java |  4 +-
 .../network/shuffle/ExternalBlockHandler.java  | 27 +++-
 .../shuffle/RetryingBlockTransferorSuite.java  |  8 +--
 .../network/yarn/YarnShuffleServiceMetrics.java| 24 +++
 .../apache/spark/util/sketch/BloomFilterImpl.java  | 16 ++---
 .../spark/util/sketch/CountMinSketchImpl.java  | 16 ++---
 .../java/org/apache/spark/util/sketch/Utils.java   | 16 ++---
 .../org/apache/spark/unsafe/types/UTF8String.java  |  6 +-
 .../org/apache/spark/io/ReadAheadInputStream.java  |  4 +-
 .../unsafe/sort/UnsafeExternalSorter.java  |  4 +-
 .../unsafe/sort/UnsafeInMemorySorter.java  |  5 +-
 .../org/apache/spark/launcher/LauncherServer.java  |  4 +-
 .../expressions/SpecializedGettersReader.java  |  8 +--
 .../sql/catalyst/expressions/UnsafeDataUtils.java  |  6 +-
 .../spark/sql/catalyst/expressions/UnsafeRow.java  | 12 ++--
 .../spark/sql/connector/read/streaming/Offset.java |  4 +-
 .../sql/connector/util/V2ExpressionSQLBuilder.java | 17 ++---
 .../spark/sql/vectorized/ArrowColumnVector.java| 76 +++---
 .../spark/sql/vectorized/ColumnarBatchRow.java |  8 +--
 .../apache/spark/sql/vectorized/ColumnarRow.java   |  4 +-
 .../datasources/orc/OrcAtomicColumnVector.java | 20 +++---
 .../execution/datasources/orc/OrcFooterReader.java | 14 ++--
 .../parquet/ParquetVectorUpdaterFactory.java   |  9 ++-
 .../parquet/VectorizedColumnReader.java|  7 +-
 .../execution/vectorized/ConstantColumnVector.java |  4 +-
 .../execution/vectorized/MutableColumnarRow.java   |  4 +-
 .../JavaAdvancedDataSourceV2WithV2Filter.jav

(spark) branch master updated: [SPARK-46186][CONNECT][TESTS][FOLLOWUP] Remove flakiness of `ReattachableExecuteSuite`

2023-12-06 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 982c7268f4c [SPARK-46186][CONNECT][TESTS][FOLLOWUP] Remove flakiness 
of `ReattachableExecuteSuite`
982c7268f4c is described below

commit 982c7268f4c7ea1fa03ea679146f9e83f31bece7
Author: Juliusz Sompolski 
AuthorDate: Wed Dec 6 10:57:45 2023 -0800

[SPARK-46186][CONNECT][TESTS][FOLLOWUP] Remove flakiness of 
`ReattachableExecuteSuite`

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

The test added in https://github.com/apache/spark/pull/44095 could be flaky 
because `MEDIUM_RESULTS_QUERY` could very quickly finish before interrupt was 
sent. Replace it with a query that sleeps 30 seconds, so that we are sure that 
interrupt runs before it finishes.

### Why are the changes needed?

Remove test flakiness.

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

No.

### How was this patch tested?

Rerun ReattachableExecuteSuite 100+ times to check it isn't flaky.

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

Github Copilot was assisting in some boilerplate auto-completion.

Generated-by: Github Copilot

Closes #44189 from juliuszsompolski/SPARK-46186-followup.

Authored-by: Juliusz Sompolski 
Signed-off-by: Dongjoon Hyun 
---
 .../sql/connect/execution/ReattachableExecuteSuite.scala | 12 +++-
 1 file changed, 11 insertions(+), 1 deletion(-)

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 02b75f04495..f80229c6198 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
@@ -298,6 +298,15 @@ class ReattachableExecuteSuite extends 
SparkConnectServerTest {
   }
 
   test("SPARK-46186 interrupt directly after query start") {
+// register a sleep udf in the session
+val serverSession =
+  SparkConnectService.getOrCreateIsolatedSession(defaultUserId, 
defaultSessionId).session
+serverSession.udf.register(
+  "sleep",
+  ((ms: Int) => {
+Thread.sleep(ms);
+ms
+  }))
 // This test depends on fast timing.
 // If something is wrong, it can fail only from time to time.
 withRawBlockingStub { stub =>
@@ -309,12 +318,13 @@ class ReattachableExecuteSuite extends 
SparkConnectServerTest {
 .setOperationId(operationId)
 .build()
   val iter = stub.executePlan(
-buildExecutePlanRequest(buildPlan(MEDIUM_RESULTS_QUERY), operationId = 
operationId))
+buildExecutePlanRequest(buildPlan("select sleep(3) as s"), 
operationId = operationId))
   // wait for execute holder to exist, but the execute thread may not have 
started yet.
   Eventually.eventually(timeout(eventuallyTimeout)) {
 assert(SparkConnectService.executionManager.listExecuteHolders.length 
== 1)
   }
   stub.interrupt(interruptRequest)
+  // make sure the client gets the OPERATION_CANCELED error
   val e = intercept[StatusRuntimeException] {
 while (iter.hasNext) iter.next()
   }


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



(spark) branch master updated: [SPARK-45580][SQL] Handle case where a nested subquery becomes an existence join

2023-12-06 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 c96fef2ea55 [SPARK-45580][SQL] Handle case where a nested subquery 
becomes an existence join
c96fef2ea55 is described below

commit c96fef2ea55ee85ac66905584e9dee31471de9f1
Author: Bruce Robbins 
AuthorDate: Wed Dec 6 10:55:15 2023 -0800

[SPARK-45580][SQL] Handle case where a nested subquery becomes an existence 
join

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

In `RewritePredicateSubquery`, prune existence flags from the final join 
when `rewriteExistentialExpr` returns an existence join. This change prunes the 
flags (attributes with the name "exists") by adding a `Project` node.

For example:
```
Join LeftSemi, ((a#13 = c1#15) OR exists#19)
:- Join ExistenceJoin(exists#19), (a#13 = col1#17)
:  :- LocalRelation [a#13]
:  +- LocalRelation [col1#17]
+- LocalRelation [c1#15]
```
becomes
```
Project [a#13]
+- Join LeftSemi, ((a#13 = c1#15) OR exists#19)
   :- Join ExistenceJoin(exists#19), (a#13 = col1#17)
   :  :- LocalRelation [a#13]
   :  +- LocalRelation [col1#17]
   +- LocalRelation [c1#15]
```
This change always adds the `Project` node, whether 
`rewriteExistentialExpr` returns an existence join or not. In the case when 
`rewriteExistentialExpr` does not return an existence join, 
`RemoveNoopOperators` will remove the unneeded `Project` node.

### Why are the changes needed?

This query returns an extraneous boolean column when run in spark-sql:
```
create or replace temp view t1(a) as values (1), (2), (3), (7);
create or replace temp view t2(c1) as values (1), (2), (3);
create or replace temp view t3(col1) as values (3), (9);

select *
from t1
where exists (
  select c1
  from t2
  where a = c1
  or a in (select col1 from t3)
);

1   false
2   false
3   true
```
(Note: the above query will not have the extraneous boolean column when run 
from the Dataset API. That is because the Dataset API truncates the rows based 
on the schema of the analyzed plan. The bug occurs during optimization).

This query fails when run in either spark-sql or using the Dataset API:
```
select (
  select *
  from t1
  where exists (
select c1
from t2
where a = c1
or a in (select col1 from t3)
  )
  limit 1
)
from range(1);

java.lang.AssertionError: assertion failed: Expects 1 field, but got 2; 
something went wrong in analysis
```

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

No, except for the removal of the extraneous boolean flag and the fix to 
the error condition.

### How was this patch tested?

New unit test.

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

No.

Closes #44193 from bersprockets/schema_change.

Authored-by: Bruce Robbins 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/sql/catalyst/optimizer/subquery.scala|  9 +++--
 .../scala/org/apache/spark/sql/SubquerySuite.scala | 46 ++
 2 files changed, 52 insertions(+), 3 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
index 1f1a16e9093..6ca2cb79aaf 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala
@@ -132,19 +132,22 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] 
with PredicateHelper {
   withSubquery.foldLeft(newFilter) {
 case (p, Exists(sub, _, _, conditions, subHint)) =>
   val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, rewriteDomainJoinsIfPresent(outerPlan, sub, 
joinCond),
+  val join = buildJoin(outerPlan, 
rewriteDomainJoinsIfPresent(outerPlan, sub, joinCond),
 LeftSemi, joinCond, subHint)
+  Project(p.output, join)
 case (p, Not(Exists(sub, _, _, conditions, subHint))) =>
   val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p)
-  buildJoin(outerPlan, rewriteDomainJoinsIfPresent(outerPlan, sub, 
joinCond),
+  val join = buildJoin(outerPlan, 
rewriteDomainJoinsIfPresent(outerPlan, sub, joinCond),
 LeftAnti, joinCond, subHint)
+  Project(p.output, join)
 case (p, InSubquery(values, ListQuery(sub, _, _, _, conditions, 
subHint))) =>
   // Deduplicate conflicting attributes if any.
  

(spark) branch master updated: [SPARK-46232][PYTHON][FOLLOWUP] Migrate `ValueError` into `PySparkValueError`

2023-12-06 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 467df65ceba [SPARK-46232][PYTHON][FOLLOWUP] Migrate `ValueError` into 
`PySparkValueError`
467df65ceba is described below

commit 467df65ceba5f6a8957ca7d72f5537434bf32e81
Author: Haejoon Lee 
AuthorDate: Wed Dec 6 10:51:22 2023 -0800

[SPARK-46232][PYTHON][FOLLOWUP] Migrate `ValueError` into 
`PySparkValueError`

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

This PR followups for https://github.com/apache/spark/pull/44149 to address 
missing case.

### Why are the changes needed?

To improve error handling.

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

No API changes.

### How was this patch tested?

The existing CI should pass.

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

No.

Closes #44202 from itholic/SPARK-46232-followup.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/sql/pandas/serializers.py | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/python/pyspark/sql/pandas/serializers.py 
b/python/pyspark/sql/pandas/serializers.py
index 8b2b583ddaa..834f22c86c0 100644
--- a/python/pyspark/sql/pandas/serializers.py
+++ b/python/pyspark/sql/pandas/serializers.py
@@ -738,8 +738,9 @@ class 
CogroupPandasUDFSerializer(ArrowStreamPandasUDFSerializer):
 )
 
 elif dataframes_in_group != 0:
-raise ValueError(
-"Invalid number of pandas.DataFrames in group 
{0}".format(dataframes_in_group)
+raise PySparkValueError(
+error_class="INVALID_NUMBER_OF_DATAFRAMES_IN_GROUP",
+message_parameters={"dataframes_in_group": 
str(dataframes_in_group)},
 )
 
 


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



(spark) branch master updated: [SPARK-46283][INFRA] Remove `streaming-kinesis-asl` module from `MODULES_TO_TEST` for branch-3.x daily tests

2023-12-06 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 809cec01205 [SPARK-46283][INFRA] Remove `streaming-kinesis-asl` module 
from `MODULES_TO_TEST` for branch-3.x daily tests
809cec01205 is described below

commit 809cec012055d6f15987f338122d2fdb5bdd5c92
Author: yangjie01 
AuthorDate: Wed Dec 6 10:50:27 2023 -0800

[SPARK-46283][INFRA] Remove `streaming-kinesis-asl` module from 
`MODULES_TO_TEST` for branch-3.x daily tests

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

After the merge of https://github.com/apache/spark/pull/43736, the master 
branch began testing the `streaming-kinesis-asl` module.

At the same time, because the daily test will reuse `build_and_test.yml`, 
the daily test of branch-3.x also began testing `streaming-kinesis-asl`.

However, in branch-3.x, the env `ENABLE_KINESIS_TESTS` is hard-coded as 1 
in `dev/sparktestsupport/modules.py`:


https://github.com/apache/spark/blob/1321b4e64deaa1e58bf297c25b72319083056568/dev/sparktestsupport/modules.py#L332-L346

which leads to the failure of the daily test of branch-3.x:

- branch-3.3: https://github.com/apache/spark/actions/runs/7111246311
- branch-3.4: https://github.com/apache/spark/actions/runs/7098435892
- branch-3.5: https://github.com/apache/spark/actions/runs/7099811235

```
[info] 
org.apache.spark.streaming.kinesis.WithoutAggregationKinesisStreamSuite *** 
ABORTED *** (1 second, 14 milliseconds)
[info]   java.lang.Exception: Kinesis tests enabled using environment 
variable ENABLE_KINESIS_TESTS
[info] but could not find AWS credentials. Please follow instructions in 
AWS documentation
[info] to set the credentials in your system such that the 
DefaultAWSCredentialsProviderChain
[info] can find the credentials.
[info]   at 
org.apache.spark.streaming.kinesis.KinesisTestUtils$.getAWSCredentials(KinesisTestUtils.scala:258)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisTestUtils.kinesisClient$lzycompute(KinesisTestUtils.scala:58)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisTestUtils.kinesisClient(KinesisTestUtils.scala:57)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisTestUtils.describeStream(KinesisTestUtils.scala:168)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisTestUtils.findNonExistentStreamName(KinesisTestUtils.scala:181)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisTestUtils.createStream(KinesisTestUtils.scala:84)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisStreamTests.$anonfun$beforeAll$1(KinesisStreamSuite.scala:61)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisFunSuite.runIfTestsEnabled(KinesisFunSuite.scala:41)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisFunSuite.runIfTestsEnabled$(KinesisFunSuite.scala:39)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisStreamTests.runIfTestsEnabled(KinesisStreamSuite.scala:42)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisStreamTests.beforeAll(KinesisStreamSuite.scala:59)
[info]   at 
org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:212)
[info]   at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
[info]   at 
org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisStreamTests.org$scalatest$BeforeAndAfter$$super$run(KinesisStreamSuite.scala:42)
[info]   at org.scalatest.BeforeAndAfter.run(BeforeAndAfter.scala:273)
[info]   at org.scalatest.BeforeAndAfter.run$(BeforeAndAfter.scala:271)
[info]   at 
org.apache.spark.streaming.kinesis.KinesisStreamTests.run(KinesisStreamSuite.scala:42)
[info]   at 
org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:321)
[info]   at 
org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:517)
[info]   at sbt.ForkMain$Run.lambda$runTest$1(ForkMain.java:414)
[info]   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
[info]   at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
[info]   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
[info]   at java.lang.Thread.run(Thread.java:750)
[info] Test run 
org.apache.spark.streaming.kinesis.JavaKinesisInputDStreamBuilderSuite started
[info] Test 
org.apache.spark.streaming.kinesis.JavaKinesisInputDStreamBuilderSuite.testJavaKinesisDStreamBuilderOldApi
 started
[info] Test 
org.apache.spark.streaming.kinesis.JavaKinesisInputDStreamBuilderSuite.testJavaKinesisDStreamBuilder
 started
[info] Test run 
org.apache.spark.streaming.kinesis.JavaKinesisInputDStreamBuilderSuite 
finished: 0 failed, 0

(spark) branch branch-3.3 updated: [SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled`

2023-12-06 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 37d10ec3644 [SPARK-46286][DOCS] Document 
`spark.io.compression.zstd.bufferPool.enabled`
37d10ec3644 is described below

commit 37d10ec3644d41396cd7378fdc3fe405b680203c
Author: Kent Yao 
AuthorDate: Wed Dec 6 10:46:31 2023 -0800

[SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled`

This PR adds spark.io.compression.zstd.bufferPool.enabled to documentation

- Missing docs
- https://github.com/apache/spark/pull/31502#issuecomment-774792276 
potential regression

no

doc build

no

Closes #44207 from yaooqinn/SPARK-46286.

Authored-by: Kent Yao 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 6b6980de451e655ef4b9f63d502b73c09a513d4c)
Signed-off-by: Dongjoon Hyun 
---
 docs/configuration.md | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/configuration.md b/docs/configuration.md
index b96defb2adb..2a205522989 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1545,6 +1545,14 @@ Apart from these, the following properties are also 
available, and may be useful
   
   2.3.0
 
+
+  spark.io.compression.zstd.bufferPool.enabled
+  true
+  
+If true, enable buffer pool of ZSTD JNI library.
+  
+  3.2.0
+
 
   spark.kryo.classesToRegister
   (none)


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



(spark) branch branch-3.4 updated: [SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled`

2023-12-06 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 93fef098a0d [SPARK-46286][DOCS] Document 
`spark.io.compression.zstd.bufferPool.enabled`
93fef098a0d is described below

commit 93fef098a0d5d6c95205a46ebf9c959e325c9d7e
Author: Kent Yao 
AuthorDate: Wed Dec 6 10:46:31 2023 -0800

[SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled`

This PR adds spark.io.compression.zstd.bufferPool.enabled to documentation

- Missing docs
- https://github.com/apache/spark/pull/31502#issuecomment-774792276 
potential regression

no

doc build

no

Closes #44207 from yaooqinn/SPARK-46286.

Authored-by: Kent Yao 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 6b6980de451e655ef4b9f63d502b73c09a513d4c)
Signed-off-by: Dongjoon Hyun 
---
 docs/configuration.md | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/configuration.md b/docs/configuration.md
index 198a6dd4b2b..6bd49f398d9 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1727,6 +1727,14 @@ Apart from these, the following properties are also 
available, and may be useful
   
   2.3.0
 
+
+  spark.io.compression.zstd.bufferPool.enabled
+  true
+  
+If true, enable buffer pool of ZSTD JNI library.
+  
+  3.2.0
+
 
   spark.kryo.classesToRegister
   (none)


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



(spark) branch branch-3.5 updated: [SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled`

2023-12-06 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new b5cbe1fcdb4 [SPARK-46286][DOCS] Document 
`spark.io.compression.zstd.bufferPool.enabled`
b5cbe1fcdb4 is described below

commit b5cbe1fcdb464fc064ffb5fbef3edfa408d6638f
Author: Kent Yao 
AuthorDate: Wed Dec 6 10:46:31 2023 -0800

[SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled`

This PR adds spark.io.compression.zstd.bufferPool.enabled to documentation

- Missing docs
- https://github.com/apache/spark/pull/31502#issuecomment-774792276 
potential regression

no

doc build

no

Closes #44207 from yaooqinn/SPARK-46286.

Authored-by: Kent Yao 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 6b6980de451e655ef4b9f63d502b73c09a513d4c)
Signed-off-by: Dongjoon Hyun 
---
 docs/configuration.md | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/configuration.md b/docs/configuration.md
index 248f9333c9a..f79406c5b6d 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1752,6 +1752,14 @@ Apart from these, the following properties are also 
available, and may be useful
   
   2.3.0
 
+
+  spark.io.compression.zstd.bufferPool.enabled
+  true
+  
+If true, enable buffer pool of ZSTD JNI library.
+  
+  3.2.0
+
 
   spark.kryo.classesToRegister
   (none)


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



(spark) branch master updated: [SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled`

2023-12-06 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 6b6980de451 [SPARK-46286][DOCS] Document 
`spark.io.compression.zstd.bufferPool.enabled`
6b6980de451 is described below

commit 6b6980de451e655ef4b9f63d502b73c09a513d4c
Author: Kent Yao 
AuthorDate: Wed Dec 6 10:46:31 2023 -0800

[SPARK-46286][DOCS] Document `spark.io.compression.zstd.bufferPool.enabled`

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

This PR adds spark.io.compression.zstd.bufferPool.enabled to documentation

### Why are the changes needed?

- Missing docs
- https://github.com/apache/spark/pull/31502#issuecomment-774792276 
potential regression

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

no

### How was this patch tested?

doc build

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

no

Closes #44207 from yaooqinn/SPARK-46286.

Authored-by: Kent Yao 
Signed-off-by: Dongjoon Hyun 
---
 docs/configuration.md | 8 
 1 file changed, 8 insertions(+)

diff --git a/docs/configuration.md b/docs/configuration.md
index 2ad07cf59f7..f261e3b2deb 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -1760,6 +1760,14 @@ Apart from these, the following properties are also 
available, and may be useful
   
   2.3.0
 
+
+  spark.io.compression.zstd.bufferPool.enabled
+  true
+  
+If true, enable buffer pool of ZSTD JNI library.
+  
+  3.2.0
+
 
   spark.io.compression.zstd.workers
   0


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



(spark) branch master updated: [SPARK-46287][PYTHON][CONNECT] `DataFrame.isEmpty` should work with all datatypes

2023-12-06 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 f4e41e0e318 [SPARK-46287][PYTHON][CONNECT] `DataFrame.isEmpty` should 
work with all datatypes
f4e41e0e318 is described below

commit f4e41e0e318ea1269de5991f4635637e6e5233f3
Author: Ruifeng Zheng 
AuthorDate: Wed Dec 6 10:45:12 2023 -0800

[SPARK-46287][PYTHON][CONNECT] `DataFrame.isEmpty` should work with all 
datatypes

### What changes were proposed in this pull request?
`DataFrame.isEmpty` should work with all datatypes

the schema maybe not compatible with arrow, so should not use 
`collect/take` to check `isEmpty`

### Why are the changes needed?
bugfix

### Does this PR introduce _any_ user-facing change?
before:
```
In [1]: spark.sql("SELECT INTERVAL '10-8' YEAR TO MONTH AS 
interval").isEmpty()
23/12/06 20:39:58 WARN CheckAllocator: More than one 
DefaultAllocationManager on classpath. Choosing first found
--- 
/ 1]
KeyError  Traceback (most recent call last)
Cell In[1], line 1
> 1 spark.sql("SELECT INTERVAL '10-8' YEAR TO MONTH AS 
interval").isEmpty()

File ~/Dev/spark/python/pyspark/sql/connect/dataframe.py:181, in 
DataFrame.isEmpty(self)
180 def isEmpty(self) -> bool:
--> 181 return len(self.take(1)) == 0

...

File 
~/.dev/miniconda3/envs/spark_dev_311/lib/python3.11/site-packages/pyarrow/public-api.pxi:208,
 in pyarrow.lib.pyarrow_wrap_array()

File 
~/.dev/miniconda3/envs/spark_dev_311/lib/python3.11/site-packages/pyarrow/array.pxi:3659,
 in pyarrow.lib.get_array_class_from_type()

KeyError: 21
```

after
```
In [1]: spark.sql("SELECT INTERVAL '10-8' YEAR TO MONTH AS 
interval").isEmpty()
23/12/06 20:40:26 WARN CheckAllocator: More than one 
DefaultAllocationManager on classpath. Choosing first found
Out[1]: False
```

### How was this patch tested?
added ut

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

Closes #44209 from zhengruifeng/py_connect_df_isempty.

Authored-by: Ruifeng Zheng 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/sql/connect/dataframe.py| 2 +-
 python/pyspark/sql/tests/connect/test_connect_basic.py | 5 +
 2 files changed, 6 insertions(+), 1 deletion(-)

diff --git a/python/pyspark/sql/connect/dataframe.py 
b/python/pyspark/sql/connect/dataframe.py
index 6a1d4571216..66059ad96eb 100644
--- a/python/pyspark/sql/connect/dataframe.py
+++ b/python/pyspark/sql/connect/dataframe.py
@@ -178,7 +178,7 @@ class DataFrame:
 write.__doc__ = PySparkDataFrame.write.__doc__
 
 def isEmpty(self) -> bool:
-return len(self.take(1)) == 0
+return len(self.select().take(1)) == 0
 
 isEmpty.__doc__ = PySparkDataFrame.isEmpty.__doc__
 
diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py 
b/python/pyspark/sql/tests/connect/test_connect_basic.py
index fb5eaece7f4..5e0cf535391 100755
--- a/python/pyspark/sql/tests/connect/test_connect_basic.py
+++ b/python/pyspark/sql/tests/connect/test_connect_basic.py
@@ -2004,6 +2004,11 @@ class SparkConnectBasicTests(SparkConnectSQLTestCase):
 self.assertFalse(self.connect.sql("SELECT 1 AS X").isEmpty())
 self.assertTrue(self.connect.sql("SELECT 1 AS X LIMIT 0").isEmpty())
 
+def test_is_empty_with_unsupported_types(self):
+df = self.spark.sql("SELECT INTERVAL '10-8' YEAR TO MONTH AS interval")
+self.assertEqual(df.count(), 1)
+self.assertFalse(df.isEmpty())
+
 def test_session(self):
 self.assertEqual(self.connect, self.connect.sql("SELECT 
1").sparkSession)
 


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



(spark) branch master updated: [SPARK-46288][PS][TESTS] Remove unused code in `pyspark.pandas.tests.frame.*`

2023-12-06 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 f6861c3918b [SPARK-46288][PS][TESTS] Remove unused code in 
`pyspark.pandas.tests.frame.*`
f6861c3918b is described below

commit f6861c3918bdedf5d8d89dbecced3317cc9dc490
Author: Ruifeng Zheng 
AuthorDate: Wed Dec 6 10:44:04 2023 -0800

[SPARK-46288][PS][TESTS] Remove unused code in 
`pyspark.pandas.tests.frame.*`

### What changes were proposed in this pull request?
Remove unused code in `pyspark.pandas.tests.frame.*`

### Why are the changes needed?
code clean up

### Does this PR introduce _any_ user-facing change?
no, test-only

### How was this patch tested?
ci

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

Closes #44212 from zhengruifeng/ps_frame_cleanup.

Authored-by: Ruifeng Zheng 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/pandas/tests/frame/test_conversion.py  |  6 --
 python/pyspark/pandas/tests/frame/test_reindexing.py  | 13 -
 python/pyspark/pandas/tests/frame/test_spark.py   |  6 --
 python/pyspark/pandas/tests/frame/test_take.py| 14 --
 python/pyspark/pandas/tests/frame/test_time_series.py | 13 -
 python/pyspark/pandas/tests/frame/test_truncate.py| 14 --
 6 files changed, 66 deletions(-)

diff --git a/python/pyspark/pandas/tests/frame/test_conversion.py 
b/python/pyspark/pandas/tests/frame/test_conversion.py
index 116a7d31c11..eefb461239e 100644
--- a/python/pyspark/pandas/tests/frame/test_conversion.py
+++ b/python/pyspark/pandas/tests/frame/test_conversion.py
@@ -34,12 +34,6 @@ class FrameConversionMixin:
 index=np.random.rand(9),
 )
 
-@property
-def df_pair(self):
-pdf = self.pdf
-psdf = ps.from_pandas(pdf)
-return pdf, psdf
-
 def test_astype(self):
 psdf = self.psdf
 
diff --git a/python/pyspark/pandas/tests/frame/test_reindexing.py 
b/python/pyspark/pandas/tests/frame/test_reindexing.py
index 606efd95188..b3639945391 100644
--- a/python/pyspark/pandas/tests/frame/test_reindexing.py
+++ b/python/pyspark/pandas/tests/frame/test_reindexing.py
@@ -30,19 +30,6 @@ from pyspark.testing.sqlutils import SQLTestUtils
 # This file contains test cases for 'Reindexing / Selection / Label 
manipulation'
 # 
https://spark.apache.org/docs/latest/api/python/reference/pyspark.pandas/frame.html#reindexing-selection-label-manipulation
 class FrameReindexingMixin:
-@property
-def pdf(self):
-return pd.DataFrame(
-{"a": [1, 2, 3, 4, 5, 6, 7, 8, 9], "b": [4, 5, 6, 3, 2, 1, 0, 0, 
0]},
-index=np.random.rand(9),
-)
-
-@property
-def df_pair(self):
-pdf = self.pdf
-psdf = ps.from_pandas(pdf)
-return pdf, psdf
-
 def test_add_prefix(self):
 pdf = pd.DataFrame({"A": [1, 2, 3, 4], "B": [3, 4, 5, 6]}, 
index=np.random.rand(4))
 psdf = ps.from_pandas(pdf)
diff --git a/python/pyspark/pandas/tests/frame/test_spark.py 
b/python/pyspark/pandas/tests/frame/test_spark.py
index 4413279e32f..36466695c30 100644
--- a/python/pyspark/pandas/tests/frame/test_spark.py
+++ b/python/pyspark/pandas/tests/frame/test_spark.py
@@ -43,12 +43,6 @@ class FrameSparkMixin:
 index=np.random.rand(9),
 )
 
-@property
-def df_pair(self):
-pdf = self.pdf
-psdf = ps.from_pandas(pdf)
-return pdf, psdf
-
 def test_empty_dataframe(self):
 pdf = pd.DataFrame({"a": pd.Series([], dtype="i1"), "b": pd.Series([], 
dtype="str")})
 
diff --git a/python/pyspark/pandas/tests/frame/test_take.py 
b/python/pyspark/pandas/tests/frame/test_take.py
index 28d20e9bd99..3654436848b 100644
--- a/python/pyspark/pandas/tests/frame/test_take.py
+++ b/python/pyspark/pandas/tests/frame/test_take.py
@@ -16,7 +16,6 @@
 #
 import unittest
 
-import numpy as np
 import pandas as pd
 
 from pyspark import pandas as ps
@@ -25,19 +24,6 @@ from pyspark.testing.sqlutils import SQLTestUtils
 
 
 class FrameTakeMixin:
-@property
-def pdf(self):
-return pd.DataFrame(
-{"a": [1, 2, 3, 4, 5, 6, 7, 8, 9], "b": [4, 5, 6, 3, 2, 1, 0, 0, 
0]},
-index=np.random.rand(9),
-)
-
-@property
-def df_pair(self):
-pdf = self.pdf
-psdf = ps.from_pandas(pdf)
-return pdf, psdf
-
 def test_take(self):
 pdf = pd.DataFrame(
 {"A": range(0, 5), "B": range(10, 0, -2), "C": 
range(10, 5, -1)}
diff --git a/python/pyspark/pandas/tests/frame/test_time_series.py 
b/python/pyspark/pandas/tests/frame/

(spark) branch master updated: [SPARK-46268][PS][CONNECT][TESTS] Re-organize `StatsTests`

2023-12-05 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 ef05fb632424 [SPARK-46268][PS][CONNECT][TESTS] Re-organize `StatsTests`
ef05fb632424 is described below

commit ef05fb632424f8f121bcd4518ab3a8815c295c85
Author: Ruifeng Zheng 
AuthorDate: Tue Dec 5 10:19:41 2023 -0800

[SPARK-46268][PS][CONNECT][TESTS] Re-organize `StatsTests`

### What changes were proposed in this pull request?
Re-organize `StatsTests`

### Why are the changes needed?
break the big test file by grouping test cases by topics

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

### How was this patch tested?
ci

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

Closes #44185 from zhengruifeng/ps_reorg_test_stats.

Authored-by: Ruifeng Zheng 
Signed-off-by: Dongjoon Hyun 
---
 dev/sparktestsupport/modules.py|   8 +-
 .../pyspark/pandas/tests/computation/test_corr.py  | 222 
 .../pandas/tests/{ => computation}/test_stats.py   | 278 +
 .../test_parity_corr.py}   |   7 +-
 .../connect/{ => computation}/test_parity_stats.py |   4 +-
 .../test_parity_axis.py}   |   6 +-
 python/pyspark/pandas/tests/frame/test_axis.py | 135 ++
 7 files changed, 373 insertions(+), 287 deletions(-)

diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index 9bbe86baa1dc..900329d07c00 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -789,6 +789,7 @@ pyspark_pandas_slow = Module(
 "pyspark.pandas.tests.computation.test_binary_ops",
 "pyspark.pandas.tests.computation.test_combine",
 "pyspark.pandas.tests.computation.test_compute",
+"pyspark.pandas.tests.computation.test_corr",
 "pyspark.pandas.tests.computation.test_corrwith",
 "pyspark.pandas.tests.computation.test_cov",
 "pyspark.pandas.tests.computation.test_cumulative",
@@ -797,7 +798,9 @@ pyspark_pandas_slow = Module(
 "pyspark.pandas.tests.computation.test_melt",
 "pyspark.pandas.tests.computation.test_missing_data",
 "pyspark.pandas.tests.computation.test_pivot",
+"pyspark.pandas.tests.computation.test_stats",
 "pyspark.pandas.tests.frame.test_attrs",
+"pyspark.pandas.tests.frame.test_axis",
 "pyspark.pandas.tests.frame.test_constructor",
 "pyspark.pandas.tests.frame.test_conversion",
 "pyspark.pandas.tests.frame.test_reindexing",
@@ -841,7 +844,6 @@ pyspark_pandas_slow = Module(
 "pyspark.pandas.tests.series.test_series",
 "pyspark.pandas.tests.series.test_sort",
 "pyspark.pandas.tests.series.test_stat",
-"pyspark.pandas.tests.test_stats",
 ],
 excluded_python_implementations=[
 "PyPy"  # Skip these tests under PyPy since they require numpy, 
pandas, and pyarrow and
@@ -1014,6 +1016,7 @@ pyspark_pandas_connect_part0 = Module(
 "pyspark.pandas.tests.connect.computation.test_parity_combine",
 "pyspark.pandas.tests.connect.computation.test_parity_compute",
 "pyspark.pandas.tests.connect.computation.test_parity_cov",
+"pyspark.pandas.tests.connect.computation.test_parity_corr",
 "pyspark.pandas.tests.connect.computation.test_parity_corrwith",
 "pyspark.pandas.tests.connect.computation.test_parity_cumulative",
 "pyspark.pandas.tests.connect.computation.test_parity_describe",
@@ -1021,6 +1024,7 @@ pyspark_pandas_connect_part0 = Module(
 "pyspark.pandas.tests.connect.computation.test_parity_melt",
 "pyspark.pandas.tests.connect.groupby.test_parity_stat",
 "pyspark.pandas.tests.connect.frame.test_parity_attrs",
+"pyspark.pandas.tests.connect.frame.test_parity_axis",
 "pyspark.pandas.tests.connect.diff_frames_ops.test_parity_dot_frame",
 "pyspark.pandas.tests.connect.diff_frames_ops.test_parity_dot_series",
 ],
@@ -1075,7 +1079,6 @@ pyspark_pandas_connect_part1 = Module(
 "pyspark.pandas.tests.connect.series.test_parity_stat",
 
"pyspark.pandas.tests.connect.data_type_ops.test_parity_num_arithmetic",
 "pyspark.pandas.tests.connect.test_parity_reshape",
-"pyspark.pandas.tests.connect.test_parity_stats",
 
&quo

(spark) branch master updated: [SPARK-46271][PS][TESTS] Remove unused properties in `pyspark.pandas.tests.groupby.*`

2023-12-05 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 a56ca8a673d7 [SPARK-46271][PS][TESTS] Remove unused properties in 
`pyspark.pandas.tests.groupby.*`
a56ca8a673d7 is described below

commit a56ca8a673d7a0ce85a5f01e51c76d8dae81804a
Author: Ruifeng Zheng 
AuthorDate: Tue Dec 5 10:17:19 2023 -0800

[SPARK-46271][PS][TESTS] Remove unused properties in 
`pyspark.pandas.tests.groupby.*`

### What changes were proposed in this pull request?
remove unused properties in `pyspark.pandas.tests.groupby.*`

### Why are the changes needed?
just code clean up

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

### How was this patch tested?
ci

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

Closes #44188 from zhengruifeng/ps_test_cleanup.

Authored-by: Ruifeng Zheng 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/pandas/tests/groupby/test_aggregate.py| 15 ---
 python/pyspark/pandas/tests/groupby/test_apply_func.py   | 15 ---
 python/pyspark/pandas/tests/groupby/test_cumulative.py   | 15 ---
 python/pyspark/pandas/tests/groupby/test_describe.py | 15 ---
 python/pyspark/pandas/tests/groupby/test_groupby.py  | 15 ---
 python/pyspark/pandas/tests/groupby/test_head_tail.py| 15 ---
 python/pyspark/pandas/tests/groupby/test_index.py| 15 ---
 python/pyspark/pandas/tests/groupby/test_missing_data.py | 15 ---
 python/pyspark/pandas/tests/groupby/test_split_apply.py  | 15 ---
 9 files changed, 135 deletions(-)

diff --git a/python/pyspark/pandas/tests/groupby/test_aggregate.py 
b/python/pyspark/pandas/tests/groupby/test_aggregate.py
index 9e7f3f6cd113..f6fb37aa9824 100644
--- a/python/pyspark/pandas/tests/groupby/test_aggregate.py
+++ b/python/pyspark/pandas/tests/groupby/test_aggregate.py
@@ -24,21 +24,6 @@ from pyspark.testing.sqlutils import SQLTestUtils
 
 
 class GroupbyAggregateMixin:
-@property
-def pdf(self):
-return pd.DataFrame(
-{
-"A": [1, 2, 1, 2],
-"B": [3.1, 4.1, 4.1, 3.1],
-"C": ["a", "b", "b", "a"],
-"D": [True, False, False, True],
-}
-)
-
-@property
-def psdf(self):
-return ps.from_pandas(self.pdf)
-
 def test_aggregate(self):
 pdf = pd.DataFrame(
 {"A": [1, 1, 2, 2], "B": [1, 2, 3, 4], "C": [0.362, 0.227, 1.267, 
-0.562]}
diff --git a/python/pyspark/pandas/tests/groupby/test_apply_func.py 
b/python/pyspark/pandas/tests/groupby/test_apply_func.py
index e7a30ff57b41..d886d8799444 100644
--- a/python/pyspark/pandas/tests/groupby/test_apply_func.py
+++ b/python/pyspark/pandas/tests/groupby/test_apply_func.py
@@ -26,21 +26,6 @@ from pyspark.testing.sqlutils import SQLTestUtils
 
 
 class GroupbyApplyFuncMixin:
-@property
-def pdf(self):
-return pd.DataFrame(
-{
-"A": [1, 2, 1, 2],
-"B": [3.1, 4.1, 4.1, 3.1],
-"C": ["a", "b", "b", "a"],
-"D": [True, False, False, True],
-}
-)
-
-@property
-def psdf(self):
-return ps.from_pandas(self.pdf)
-
 def test_apply(self):
 pdf = pd.DataFrame(
 {"a": [1, 2, 3, 4, 5, 6], "b": [1, 1, 2, 3, 5, 8], "c": [1, 4, 9, 
16, 25, 36]},
diff --git a/python/pyspark/pandas/tests/groupby/test_cumulative.py 
b/python/pyspark/pandas/tests/groupby/test_cumulative.py
index 64d6812c7bcf..2bcc1ade2d06 100644
--- a/python/pyspark/pandas/tests/groupby/test_cumulative.py
+++ b/python/pyspark/pandas/tests/groupby/test_cumulative.py
@@ -26,21 +26,6 @@ from pyspark.testing.sqlutils import SQLTestUtils
 
 
 class GroupbyCumulativeMixin:
-@property
-def pdf(self):
-return pd.DataFrame(
-{
-"A": [1, 2, 1, 2],
-"B": [3.1, 4.1, 4.1, 3.1],
-"C": ["a", "b", "b", "a"],
-"D": [True, False, False, True],
-}
-)
-
-@property
-def psdf(self):
-return ps.from_pandas(self.pdf)
-
 def test_cumcount(self):
 pdf = pd.DataFrame(
 {
diff --git a/python/pyspark/pandas/tests/groupby/test_describe.py 
b/python/pyspark/pandas/tests/groupby/test_describe.py
index dcd0e33a3811..ae12aae61ffc 100644
--- a/python/pyspark/pandas/tests

(spark) branch master updated: [SPARK-46266][PS][CONNECT][TESTS] Re-organize `NumOpsTests`

2023-12-05 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 e9ba896f8446 [SPARK-46266][PS][CONNECT][TESTS] Re-organize 
`NumOpsTests`
e9ba896f8446 is described below

commit e9ba896f8446885d42bcb27953fe2d6050794be1
Author: Ruifeng Zheng 
AuthorDate: Tue Dec 5 08:53:05 2023 -0800

[SPARK-46266][PS][CONNECT][TESTS] Re-organize `NumOpsTests`

### What changes were proposed in this pull request?
Re-organize `NumOpsTests`, factor out the `astype` tests

### Why are the changes needed?
group the tests by topics

### Does this PR introduce _any_ user-facing change?
no, test-only

### How was this patch tested?
ci

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

Closes #44182 from zhengruifeng/ps_reorg_as_type.

Authored-by: Ruifeng Zheng 
Signed-off-by: Dongjoon Hyun 
---
 dev/sparktestsupport/modules.py|   2 +
 .../connect/data_type_ops/test_parity_as_type.py   |  43 +
 .../pandas/tests/data_type_ops/test_as_type.py | 103 +
 .../pandas/tests/data_type_ops/test_num_ops.py |  57 
 4 files changed, 148 insertions(+), 57 deletions(-)

diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index f35c42d11e58..9bbe86baa1dc 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -698,6 +698,7 @@ pyspark_pandas = Module(
 "pyspark.pandas.spark.utils",
 "pyspark.pandas.typedef.typehints",
 # unittests
+"pyspark.pandas.tests.data_type_ops.test_as_type",
 "pyspark.pandas.tests.data_type_ops.test_base",
 "pyspark.pandas.tests.data_type_ops.test_binary_ops",
 "pyspark.pandas.tests.data_type_ops.test_boolean_ops",
@@ -952,6 +953,7 @@ pyspark_pandas_connect_part0 = Module(
 ],
 python_test_goals=[
 # pandas-on-Spark unittests
+"pyspark.pandas.tests.connect.data_type_ops.test_parity_as_type",
 "pyspark.pandas.tests.connect.data_type_ops.test_parity_base",
 "pyspark.pandas.tests.connect.data_type_ops.test_parity_binary_ops",
 "pyspark.pandas.tests.connect.data_type_ops.test_parity_boolean_ops",
diff --git 
a/python/pyspark/pandas/tests/connect/data_type_ops/test_parity_as_type.py 
b/python/pyspark/pandas/tests/connect/data_type_ops/test_parity_as_type.py
new file mode 100644
index ..a2a9e28a5ab5
--- /dev/null
+++ b/python/pyspark/pandas/tests/connect/data_type_ops/test_parity_as_type.py
@@ -0,0 +1,43 @@
+#
+# 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.
+#
+import unittest
+
+from pyspark import pandas as ps
+from pyspark.pandas.tests.data_type_ops.test_as_type import AsTypeTestsMixin
+from pyspark.pandas.tests.connect.data_type_ops.testing_utils import 
OpsTestBase
+from pyspark.testing.pandasutils import PandasOnSparkTestUtils
+from pyspark.testing.connectutils import ReusedConnectTestCase
+
+
+class AsTypeParityTests(
+AsTypeTestsMixin, PandasOnSparkTestUtils, OpsTestBase, 
ReusedConnectTestCase
+):
+@property
+def psdf(self):
+return ps.from_pandas(self.pdf)
+
+
+if __name__ == "__main__":
+from pyspark.pandas.tests.connect.data_type_ops.test_parity_as_type import 
*  # noqa: F401
+
+try:
+import xmlrunner  # type: ignore[import]
+
+testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", 
verbosity=2)
+except ImportError:
+testRunner = None
+unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/pandas/tests/data_type_ops/test_as_type.py 
b/python/pyspark/pandas/tests/data_type_ops/test_as_type.py
new file mode 100644
index ..9d5c0d03d548
--- /dev/null
+++ b/python/pyspark/pandas/tests/data_type_ops/test_as_type.py
@@ -0,0 +1,103 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor

(spark) branch master updated (7c741c8c25fc -> 9808ad3ac322)

2023-12-05 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 7c741c8c25fc [SPARK-46265][CONNECT] Assertions in AddArtifact RPC make 
the connect client incompatible with older clusters
 add 9808ad3ac322 [SPARK-46259][PYTHON][DOCS] Add an appropriate link for 
error class usage documentation

No new revisions were added by this update.

Summary of changes:
 python/docs/source/development/contributing.rst | 4 ++--
 1 file changed, 2 insertions(+), 2 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-46219][SQL] Unwrap cast in join predicates

2023-12-05 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 8235f1d56bf2 [SPARK-46219][SQL] Unwrap cast in join predicates
8235f1d56bf2 is described below

commit 8235f1d56bf232bb713fe24ff6f2ffdaf49d2fcc
Author: Yuming Wang 
AuthorDate: Tue Dec 5 08:37:34 2023 -0800

[SPARK-46219][SQL] Unwrap cast in join predicates

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

In a large data platform, it is very common to join different data types. 
Similar to 
[`reorderJoinPredicates`](https://github.com/apache/spark/blob/b03afa7bde5a050eb95284b275eae0aac2257f63/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala#L321-L338).
 This PR adds a function in `EnsureRequirements` to unwrap cast in join 
predicates to reduce shuffle if they are integral types.

The key idea here is that casting to either of these two types will not 
affect the result of join for integral types join keys. For example: `a.intCol 
= try_cast(b.bigIntCol AS int)`, if the value of `bigIntCol` exceeds the range 
of int, the result of `try_cast(b.bigIntCol AS int)` is `null`, and the result 
of  `a.intCol = try_cast(b.bigIntCol AS int)` in the join condition is `false`. 
The result is consistent with `cast(a.intCol AS bigint) = b.bigIntCol`.

### Why are the changes needed?

Reduce shuffle to improve query performance.
Case 1: Shuffle before join
```sql
CREATE TABLE t1(id int) USING parquet;
CREATE TABLE t2(id int) USING parquet;
CREATE TABLE t3(id bigint) USING parquet;
SET spark.sql.autoBroadcastJoinThreshold=-1;
explain SELECT * FROM t1 JOIN t2 ON t1.id = t2.id JOIN t3 ON t1.id = t3.id;
explain SELECT * FROM (SELECT *, row_number() OVER (PARTITION BY id ORDER 
BY id) AS rn FROM t1) t JOIN t2 ON t.id = t2.id WHERE rn = 1;
```
The plan differences after this PR:
```diff
 == Physical Plan ==
 AdaptiveSparkPlan isFinalPlan=false
-+- SortMergeJoin [cast(id#10 as bigint)], [id#12L], Inner
-   :- Sort [cast(id#10 as bigint) ASC NULLS FIRST], false, 0
-   :  +- Exchange hashpartitioning(cast(id#10 as bigint), 5), 
ENSURE_REQUIREMENTS, [plan_id=54]
-   : +- SortMergeJoin [id#10], [id#11], Inner
-   ::- Sort [id#10 ASC NULLS FIRST], false, 0
-   ::  +- Exchange hashpartitioning(id#10, 5), 
ENSURE_REQUIREMENTS, [plan_id=47]
-   :: +- Filter isnotnull(id#10)
-   ::+- FileScan parquet spark_catalog.default.t1[id#10]
-   :+- Sort [id#11 ASC NULLS FIRST], false, 0
-   :   +- Exchange hashpartitioning(id#11, 5), 
ENSURE_REQUIREMENTS, [plan_id=48]
-   :  +- Filter isnotnull(id#11)
-   : +- FileScan parquet spark_catalog.default.t2[id#11]
-   +- Sort [id#12L ASC NULLS FIRST], false, 0
-  +- Exchange hashpartitioning(id#12L, 5), ENSURE_REQUIREMENTS, 
[plan_id=55]
- +- Filter isnotnull(id#12L)
-+- FileScan parquet spark_catalog.default.t3[id#12L]
++- SortMergeJoin [id#20], [try_cast(id#22L as int)], Inner
+   :- SortMergeJoin [id#20], [id#21], Inner
+   :  :- Sort [id#20 ASC NULLS FIRST], false, 0
+   :  :  +- Exchange hashpartitioning(id#20, 5), ENSURE_REQUIREMENTS, 
[plan_id=50]
+   :  : +- Filter isnotnull(id#20)
+   :  :+- FileScan parquet spark_catalog.default.t1[id#20]
+   :  +- Sort [id#21 ASC NULLS FIRST], false, 0
+   : +- Exchange hashpartitioning(id#21, 5), ENSURE_REQUIREMENTS, 
[plan_id=51]
+   :+- Filter isnotnull(id#21)
+   :   +- FileScan parquet spark_catalog.default.t2[id#21]
+   +- Sort [try_cast(id#22L as int) ASC NULLS FIRST], false, 0
+  +- Exchange hashpartitioning(try_cast(id#22L as int), 5), 
ENSURE_REQUIREMENTS, [plan_id=58]
+ +- Filter isnotnull(id#22L)
++- FileScan parquet spark_catalog.default.t3[id#22L]
```

```diff
 == Physical Plan ==
 AdaptiveSparkPlan isFinalPlan=false
-+- SortMergeJoin [cast(id#22 as bigint)], [id#23L], Inner
-   :- Sort [cast(id#22 as bigint) ASC NULLS FIRST], false, 0
-   :  +- Exchange hashpartitioning(cast(id#22 as bigint), 5), 
ENSURE_REQUIREMENTS, [plan_id=62]
-   : +- Filter (rn#20 = 1)
-   :+- Window [row_number() windowspecdefinition(id#22, id#22 ASC 
NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), 
currentrow$())) AS rn#20], [id#22], [id#22 ASC NULLS FIRST]
-   :   +- WindowGroupLimit [id#22], [id#22 ASC NULLS FIRST], 
row_number(), 1, Final
-   :  +- Sort [id#22 ASC NULLS FIRST, id#22 ASC NULLS FIRST], 
false, 0
-   : +- Exchange hashpartitioning(id#22, 5

(spark) branch master updated (b4bf9ec065bf -> c8451b58a3c4)

2023-12-05 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 b4bf9ec065bf [SPARK-46269][PS] Enable more NumPy compatibility 
function tests
 add c8451b58a3c4 [SPARK-46069][SQL][FOLLOWUP] Make sure the cast 
expression is date type when unwrap timestamp type to date type

No new revisions were added by this update.

Summary of changes:
 .../sql/catalyst/optimizer/UnwrapCastInBinaryComparison.scala  | 7 ---
 1 file changed, 4 insertions(+), 3 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-46269][PS] Enable more NumPy compatibility function tests

2023-12-05 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 b4bf9ec065bf [SPARK-46269][PS] Enable more NumPy compatibility 
function tests
b4bf9ec065bf is described below

commit b4bf9ec065bf3538e3780a833b6c5dec7b602cf2
Author: Haejoon Lee 
AuthorDate: Tue Dec 5 08:33:38 2023 -0800

[SPARK-46269][PS] Enable more NumPy compatibility function tests

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

This PR proposes to enable NumPy compatibility function tests for `arccos`, 
`exp`, `expm1` and `floor_divide`.

### Why are the changes needed?

To improve the test coverage for NumPy compatibility function.

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

No, this improves the test coverage.

### How was this patch tested?

The existing CI should pass. For flaky tests, manually ran 50 times on 
local envs.

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

No.

Closes #44186 from itholic/enable_numpy_tests.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/pandas/tests/test_numpy_compat.py | 6 +-
 1 file changed, 1 insertion(+), 5 deletions(-)

diff --git a/python/pyspark/pandas/tests/test_numpy_compat.py 
b/python/pyspark/pandas/tests/test_numpy_compat.py
index 986083d0d3af..931e5475c36f 100644
--- a/python/pyspark/pandas/tests/test_numpy_compat.py
+++ b/python/pyspark/pandas/tests/test_numpy_compat.py
@@ -26,21 +26,17 @@ from pyspark.testing.sqlutils import SQLTestUtils
 
 class NumPyCompatTestsMixin:
 blacklist = [
-# Koalas does not currently support
+# Pandas-on-Spark does not currently support
 "conj",
 "conjugate",
 "isnat",
 "matmul",
 "frexp",
 # Values are close enough but tests failed.
-"arccos",
-"exp",
-"expm1",
 "log",  # flaky
 "log10",  # flaky
 "log1p",  # flaky
 "modf",
-"floor_divide",  # flaky
 ]
 
 @property


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



(spark) branch master updated: [SPARK-46258][CORE] Add `RocksDBPersistenceEngine`

2023-12-05 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 dd11075db618 [SPARK-46258][CORE] Add `RocksDBPersistenceEngine`
dd11075db618 is described below

commit dd11075db61879e200b4121b83d4239954881ddd
Author: Dongjoon Hyun 
AuthorDate: Tue Dec 5 00:49:21 2023 -0800

[SPARK-46258][CORE] Add `RocksDBPersistenceEngine`

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

This PR aims to add `RocksDBPersistenceEngine`.

### Why are the changes needed?

To speed up `Spark Master` HA operations by **6.1x**.

```
OpenJDK 64-Bit Server VM 21.0.1+12-LTS on Linux 5.15.0-1051-azure
AMD EPYC 7763 64-Core Processor
1000 Workers:Best Time(ms)   Avg 
Time(ms)   Relative


FileSystemPersistenceEngine with JavaSerializer   1571  
 1616   3.6X
RocksDBPersistenceEngine with JavaSerializer   257  
  258  22.0X
```

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

No. This is a new backend.

### How was this patch tested?

Pass the CIs with the newly added test cases.

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

No.

Closes #44173 from dongjoon-hyun/SPARK-46258.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../PersistenceEngineBenchmark-jdk21-results.txt   |  28 +++---
 .../PersistenceEngineBenchmark-results.txt |  28 +++---
 .../org/apache/spark/deploy/master/Master.scala|   4 +
 .../spark/deploy/master/RecoveryModeFactory.scala  |  18 
 .../deploy/master/RocksDBPersistenceEngine.scala   | 103 +
 .../apache/spark/deploy/master/MasterSuite.scala   |  20 
 .../deploy/master/PersistenceEngineBenchmark.scala |  11 +++
 .../deploy/master/PersistenceEngineSuite.scala |   9 ++
 8 files changed, 195 insertions(+), 26 deletions(-)

diff --git a/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt 
b/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
index 314fb6958b69..99035eb336a3 100644
--- a/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
+++ b/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
@@ -6,18 +6,20 @@ OpenJDK 64-Bit Server VM 21.0.1+12-LTS on Linux 
5.15.0-1051-azure
 AMD EPYC 7763 64-Core Processor
 1000 Workers: Best Time(ms)   Avg 
Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   Relative
 

-ZooKeeperPersistenceEngine with JavaSerializer 5402
   5546 233  0.0 5402030.8   1.0X
-ZooKeeperPersistenceEngine with KryoSerializer 4185
   4220  32  0.0 4184623.1   1.3X
-FileSystemPersistenceEngine with JavaSerializer1591
   1634  37  0.0 1590836.4   3.4X
-FileSystemPersistenceEngine with JavaSerializer (lz4)   611
623  14  0.0  611256.6   8.8X
-FileSystemPersistenceEngine with JavaSerializer (lzf)   626
640  13  0.0  626072.2   8.6X
-FileSystemPersistenceEngine with JavaSerializer (snappy)595
628  29  0.0  594744.4   9.1X
-FileSystemPersistenceEngine with JavaSerializer (zstd)  755
774  21  0.0  754604.4   7.2X
-FileSystemPersistenceEngine with KryoSerializer 479
489   8  0.0  479404.7  11.3X
-FileSystemPersistenceEngine with KryoSerializer (lz4)   392
406  12  0.0  392165.7  13.8X
-FileSystemPersistenceEngine with KryoSerializer (lzf)   525
536  14  0.0  524916.7  10.3X
-FileSystemPersistenceEngine with KryoSerializer (snappy)519
533  14  0.0  518569.3  10.4X
-FileSystemPersistenceEngine with KryoSerializer (zstd)  627
663  31  0.0  627233.2   8.6X
-BlackHolePersistenceEngine0
  0   0  6.0 166.0   32541.8X
+ZooKeeperPersistenceEngine with JavaSerializer 5863
   6053 265  0.0 5862988.1   1.0X
+ZooKeeperPersistenceEngine with KryoSerializer 4553
   4612

(spark) branch master updated: [SPARK-46233][PYTHON] Migrate all remaining `AttributeError` into PySpark error framework

2023-12-04 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 be49ca6dd71b [SPARK-46233][PYTHON] Migrate all remaining 
`AttributeError` into PySpark error framework
be49ca6dd71b is described below

commit be49ca6dd71b87172df9d88f305f06a7b87c9ecf
Author: Haejoon Lee 
AuthorDate: Mon Dec 4 16:18:27 2023 -0800

[SPARK-46233][PYTHON] Migrate all remaining `AttributeError` into PySpark 
error framework

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

This PR proposes to migrate all remaining `AttributeError`  from 
`pyspark/sql/*` into PySpark error framework, `PySparkAttributeError` with 
assigning dedicated error classes.

### Why are the changes needed?

To improve the error handling in PySpark.

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

No API changes, but the user-facing error messages will be improved.

### How was this patch tested?

The existing CI should pass.

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

No.

Closes #44150 from itholic/migrate_attribute_error.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/sql/connect/dataframe.py | 10 +++---
 python/pyspark/sql/dataframe.py | 11 ---
 python/pyspark/sql/types.py | 13 ++---
 3 files changed, 25 insertions(+), 9 deletions(-)

diff --git a/python/pyspark/sql/connect/dataframe.py 
b/python/pyspark/sql/connect/dataframe.py
index a73a24818c0c..6a1d45712163 100644
--- a/python/pyspark/sql/connect/dataframe.py
+++ b/python/pyspark/sql/connect/dataframe.py
@@ -14,7 +14,11 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-from pyspark.errors.exceptions.base import SessionNotSameException, 
PySparkIndexError
+from pyspark.errors.exceptions.base import (
+SessionNotSameException,
+PySparkIndexError,
+PySparkAttributeError,
+)
 from pyspark.sql.connect.utils import check_dependencies
 
 check_dependencies(__name__)
@@ -1694,8 +1698,8 @@ class DataFrame:
 )
 
 if name not in self.columns:
-raise AttributeError(
-"'%s' object has no attribute '%s'" % 
(self.__class__.__name__, name)
+raise PySparkAttributeError(
+error_class="ATTRIBUTE_NOT_SUPPORTED", 
message_parameters={"attr_name": name}
 )
 
 return _to_col_with_plan_id(
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 8b40b222a289..5211d874ba33 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -43,7 +43,12 @@ from py4j.java_gateway import JavaObject, JVMView
 from pyspark import copy_func, _NoValue
 from pyspark._globals import _NoValueType
 from pyspark.context import SparkContext
-from pyspark.errors import PySparkTypeError, PySparkValueError, 
PySparkIndexError
+from pyspark.errors import (
+PySparkTypeError,
+PySparkValueError,
+PySparkIndexError,
+PySparkAttributeError,
+)
 from pyspark.rdd import (
 RDD,
 _load_from_socket,
@@ -3613,8 +3618,8 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin):
 +---+
 """
 if name not in self.columns:
-raise AttributeError(
-"'%s' object has no attribute '%s'" % 
(self.__class__.__name__, name)
+raise PySparkAttributeError(
+error_class="ATTRIBUTE_NOT_SUPPORTED", 
message_parameters={"attr_name": name}
 )
 jc = self._jdf.apply(name)
 return Column(jc)
diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py
index cbfc4ab5df02..d3eed77b3838 100644
--- a/python/pyspark/sql/types.py
+++ b/python/pyspark/sql/types.py
@@ -55,6 +55,7 @@ from pyspark.errors import (
 PySparkTypeError,
 PySparkValueError,
 PySparkIndexError,
+PySparkAttributeError,
 PySparkKeyError,
 )
 
@@ -2574,16 +2575,22 @@ class Row(tuple):
 
 def __getattr__(self, item: str) -> Any:
 if item.startswith("__"):
-raise AttributeError(item)
+raise PySparkAttributeError(
+error_class="ATTRIBUTE_NOT_SUPPORTED", 
message_parameters={"attr_name": item}
+)
 try:
 # it will be slow when it has many fields,
 # but this will not be used in normal cases
 idx = self.__fields__.index(item)
 return self[idx]
 except IndexError:
-raise AttributeError(item)
+raise PySparkAttributeError(
+error_class="ATTRIBUTE_NOT_SUPPORTED", 
me

(spark) branch branch-3.3 updated: [SPARK-46239][CORE] Hide `Jetty` info

2023-12-04 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new aaec17fb244c [SPARK-46239][CORE] Hide `Jetty` info
aaec17fb244c is described below

commit aaec17fb244c175068f4de52e1288acc6125c5e9
Author: Dongjoon Hyun 
AuthorDate: Mon Dec 4 14:41:27 2023 -0800

[SPARK-46239][CORE] Hide `Jetty` info

**What changes were proposed in this pull request?**
The PR sets parameters to hide the version of  jetty in spark.

**Why are the changes needed?**
It can avoid obtaining remote WWW service information through HTTP.

**Does this PR introduce any user-facing change?**
No

**How was this patch tested?**
Manual review

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

Closes #44158 from chenyu-opensource/branch-SPARK-46239.

Lead-authored-by: Dongjoon Hyun 
Co-authored-by: chenyu 
<119398199+chenyu-opensou...@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit ff4f59341215b7f3a87e6cd8798d49e25562fcd6)
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 ++
 1 file changed, 6 insertions(+)

diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala 
b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index 834e4dfc4841..44bbd95fad13 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -311,6 +311,12 @@ private[spark] object JettyUtils extends Logging {
   logDebug(s"Using requestHeaderSize: $requestHeaderSize")
   httpConfig.setRequestHeaderSize(requestHeaderSize)
 
+  // Hide information.
+  logDebug("Using setSendServerVersion: false")
+  httpConfig.setSendServerVersion(false)
+  logDebug("Using setSendXPoweredBy: false")
+  httpConfig.setSendXPoweredBy(false)
+
   // If SSL is configured, create the secure connector first.
   val securePort = sslOptions.createJettySslContextFactory().map { factory 
=>
 val securePort = sslOptions.port.getOrElse(if (port > 0) 
Utils.userPort(port, 400) else 0)


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



(spark) branch branch-3.4 updated: [SPARK-46239][CORE] Hide `Jetty` info

2023-12-04 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 757c3a9d62b7 [SPARK-46239][CORE] Hide `Jetty` info
757c3a9d62b7 is described below

commit 757c3a9d62b7519f5bdc50d09e472b0490b6bae8
Author: Dongjoon Hyun 
AuthorDate: Mon Dec 4 14:41:27 2023 -0800

[SPARK-46239][CORE] Hide `Jetty` info

**What changes were proposed in this pull request?**
The PR sets parameters to hide the version of  jetty in spark.

**Why are the changes needed?**
It can avoid obtaining remote WWW service information through HTTP.

**Does this PR introduce any user-facing change?**
No

**How was this patch tested?**
Manual review

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

Closes #44158 from chenyu-opensource/branch-SPARK-46239.

Lead-authored-by: Dongjoon Hyun 
Co-authored-by: chenyu 
<119398199+chenyu-opensou...@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit ff4f59341215b7f3a87e6cd8798d49e25562fcd6)
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 ++
 1 file changed, 6 insertions(+)

diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala 
b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index d8119fb94984..2407152a5498 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -312,6 +312,12 @@ private[spark] object JettyUtils extends Logging {
   logDebug(s"Using requestHeaderSize: $requestHeaderSize")
   httpConfig.setRequestHeaderSize(requestHeaderSize)
 
+  // Hide information.
+  logDebug("Using setSendServerVersion: false")
+  httpConfig.setSendServerVersion(false)
+  logDebug("Using setSendXPoweredBy: false")
+  httpConfig.setSendXPoweredBy(false)
+
   // If SSL is configured, create the secure connector first.
   val securePort = sslOptions.createJettySslContextFactory().map { factory 
=>
 val securePort = sslOptions.port.getOrElse(if (port > 0) 
Utils.userPort(port, 400) else 0)


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



(spark) branch branch-3.5 updated: [SPARK-46239][CORE] Hide `Jetty` info

2023-12-04 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 1321b4e64dea [SPARK-46239][CORE] Hide `Jetty` info
1321b4e64dea is described below

commit 1321b4e64deaa1e58bf297c25b72319083056568
Author: Dongjoon Hyun 
AuthorDate: Mon Dec 4 14:41:27 2023 -0800

[SPARK-46239][CORE] Hide `Jetty` info

**What changes were proposed in this pull request?**
The PR sets parameters to hide the version of  jetty in spark.

**Why are the changes needed?**
It can avoid obtaining remote WWW service information through HTTP.

**Does this PR introduce any user-facing change?**
No

**How was this patch tested?**
Manual review

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

Closes #44158 from chenyu-opensource/branch-SPARK-46239.

Lead-authored-by: Dongjoon Hyun 
Co-authored-by: chenyu 
<119398199+chenyu-opensou...@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit ff4f59341215b7f3a87e6cd8798d49e25562fcd6)
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 ++
 1 file changed, 6 insertions(+)

diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala 
b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index 9582bdbf5264..21753361e627 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -312,6 +312,12 @@ private[spark] object JettyUtils extends Logging {
   logDebug(s"Using requestHeaderSize: $requestHeaderSize")
   httpConfig.setRequestHeaderSize(requestHeaderSize)
 
+  // Hide information.
+  logDebug("Using setSendServerVersion: false")
+  httpConfig.setSendServerVersion(false)
+  logDebug("Using setSendXPoweredBy: false")
+  httpConfig.setSendXPoweredBy(false)
+
   // If SSL is configured, create the secure connector first.
   val securePort = sslOptions.createJettySslContextFactory().map { factory 
=>
 val securePort = sslOptions.port.getOrElse(if (port > 0) 
Utils.userPort(port, 400) else 0)


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



(spark) branch master updated: [SPARK-46239][CORE] Hide `Jetty` info

2023-12-04 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 ff4f59341215 [SPARK-46239][CORE] Hide `Jetty` info
ff4f59341215 is described below

commit ff4f59341215b7f3a87e6cd8798d49e25562fcd6
Author: Dongjoon Hyun 
AuthorDate: Mon Dec 4 14:41:27 2023 -0800

[SPARK-46239][CORE] Hide `Jetty` info

**What changes were proposed in this pull request?**
The PR sets parameters to hide the version of  jetty in spark.

**Why are the changes needed?**
It can avoid obtaining remote WWW service information through HTTP.

**Does this PR introduce any user-facing change?**
No

**How was this patch tested?**
Manual review

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

Closes #44158 from chenyu-opensource/branch-SPARK-46239.

Lead-authored-by: Dongjoon Hyun 
Co-authored-by: chenyu 
<119398199+chenyu-opensou...@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun 
---
 core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 ++
 1 file changed, 6 insertions(+)

diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala 
b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index 22adcbc32ed8..50251975d733 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -314,6 +314,12 @@ private[spark] object JettyUtils extends Logging {
   logDebug(s"Using requestHeaderSize: $requestHeaderSize")
   httpConfig.setRequestHeaderSize(requestHeaderSize)
 
+  // Hide information.
+  logDebug("Using setSendServerVersion: false")
+  httpConfig.setSendServerVersion(false)
+  logDebug("Using setSendXPoweredBy: false")
+  httpConfig.setSendXPoweredBy(false)
+
   // If SSL is configured, create the secure connector first.
   val securePort = sslOptions.createJettySslContextFactory().map { factory 
=>
 val securePort = sslOptions.port.getOrElse(if (port > 0) 
Utils.userPort(port, 400) else 0)


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



(spark) branch branch-3.3 updated: [SPARK-46092][SQL][3.3] Don't push down Parquet row group filters that overflow

2023-12-04 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new c9412307394f [SPARK-46092][SQL][3.3] Don't push down Parquet row group 
filters that overflow
c9412307394f is described below

commit c9412307394fd1a277dd7fd5b173ec34e4b123d6
Author: Johan Lasperas 
AuthorDate: Mon Dec 4 12:50:57 2023 -0800

[SPARK-46092][SQL][3.3] Don't push down Parquet row group filters that 
overflow

This is a cherry-pick from https://github.com/apache/spark/pull/44006 to 
spark 3.3

### What changes were proposed in this pull request?
This change adds a check for overflows when creating Parquet row group 
filters on an INT32 (byte/short/int) parquet type to avoid incorrectly skipping 
row groups if the predicate value doesn't fit in an INT. This can happen if the 
read schema is specified as LONG, e.g via `.schema("col LONG")`
While the Parquet readers don't support reading INT32 into a LONG, the 
overflow can lead to row groups being incorrectly skipped, bypassing the reader 
altogether and producing incorrect results instead of failing.

### Why are the changes needed?
Reading a parquet file containing INT32 values with a read schema specified 
as LONG can produce incorrect results today:
```
Seq(0).toDF("a").write.parquet(path)
spark.read.schema("a LONG").parquet(path).where(s"a < 
${Long.MaxValue}").collect()
```
will return an empty result. The correct result is either:
- Failing the query if the parquet reader doesn't support upcasting 
integers to longs (all parquet readers in Spark today)
- Return result `[0]` if the parquet reader supports that upcast (no 
readers in Spark as of now, but I'm looking into adding this capability).

### Does this PR introduce _any_ user-facing change?
The following:
```
Seq(0).toDF("a").write.parquet(path)
spark.read.schema("a LONG").parquet(path).where(s"a < 
${Long.MaxValue}").collect()
```
produces an (incorrect) empty result before this change. After this change, 
the read will fail, raising an error about the unsupported conversion from INT 
to LONG in the parquet reader.

### How was this patch tested?
- Added tests to `ParquetFilterSuite` to ensure that no row group filter is 
created when the predicate value overflows or when the value type isn't 
compatible with the parquet type
- Added test to `ParquetQuerySuite` covering the correctness issue 
described above.

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

Closes #44156 from johanl-db/SPARK-46092-row-group-skipping-overflow-3.3.

Authored-by: Johan Lasperas 
Signed-off-by: Dongjoon Hyun 
---
 .../datasources/parquet/ParquetFilters.scala   | 10 ++-
 .../datasources/parquet/ParquetFilterSuite.scala   | 71 ++
 .../datasources/parquet/ParquetQuerySuite.scala| 20 ++
 3 files changed, 99 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
index 210f37d473ad..969fbab746ad 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.execution.datasources.parquet
 
-import java.lang.{Boolean => JBoolean, Double => JDouble, Float => JFloat, 
Long => JLong}
+import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float 
=> JFloat, Long => JLong, Short => JShort}
 import java.math.{BigDecimal => JBigDecimal}
 import java.nio.charset.StandardCharsets.UTF_8
 import java.sql.{Date, Timestamp}
@@ -600,7 +600,13 @@ class ParquetFilters(
 value == null || (nameToParquetField(name).fieldType match {
   case ParquetBooleanType => value.isInstanceOf[JBoolean]
   case ParquetIntegerType if value.isInstanceOf[Period] => true
-  case ParquetByteType | ParquetShortType | ParquetIntegerType => 
value.isInstanceOf[Number]
+  case ParquetByteType | ParquetShortType | ParquetIntegerType => value 
match {
+// Byte/Short/Int are all stored as INT32 in Parquet so filters are 
built using type Int.
+// We don't create a filter if the value would overflow.
+case _: JByte | _: JShort | _: Integer => true
+case v: JLong => v.longValue() >= Int.MinValue && v.longValue() <= 
Int.MaxValue
+case _ => false
+  }
   case ParquetLongType => 

(spark) branch master updated: [SPARK-46245][CORE][SQL][SS][YARN][K8S][UI] Replcace `s.c.MapOps.view.filterKeys` with `s.c.MapOps.filter`

2023-12-04 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 ac0bd2eb7b40 [SPARK-46245][CORE][SQL][SS][YARN][K8S][UI] Replcace 
`s.c.MapOps.view.filterKeys` with `s.c.MapOps.filter`
ac0bd2eb7b40 is described below

commit ac0bd2eb7b4089096f9fb288482b2f1b5049b7e2
Author: yangjie01 
AuthorDate: Mon Dec 4 12:49:52 2023 -0800

[SPARK-46245][CORE][SQL][SS][YARN][K8S][UI] Replcace 
`s.c.MapOps.view.filterKeys` with `s.c.MapOps.filter`

### What changes were proposed in this pull request?
This pr uses `s.c.MapOps.filter` to simplify code pattern 
`s.c.MapOps.view.filterKeys`.

### Why are the changes needed?
The coding pattern of `s.c.MapOps.view.filterKeys` seems verbose, it can be 
simplified using `s.c.MapOps.filter`.

### 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 #44160 from LuciferYang/SPARK-46245.

Authored-by: yangjie01 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/sql/kafka010/KafkaContinuousStream.scala |  2 +-
 .../scala/org/apache/spark/deploy/master/Master.scala  |  3 +--
 .../spark/deploy/rest/RestSubmissionClient.scala   |  4 ++--
 .../spark/executor/CoarseGrainedExecutorBackend.scala  |  8 
 .../org/apache/spark/resource/ResourceProfile.scala|  4 ++--
 .../org/apache/spark/scheduler/DAGScheduler.scala  |  4 +++-
 .../cluster/CoarseGrainedSchedulerBackend.scala|  2 +-
 .../scheduler/cluster/StandaloneSchedulerBackend.scala |  4 ++--
 .../spark/storage/ShuffleBlockFetcherIterator.scala|  2 +-
 .../main/scala/org/apache/spark/ui/PagedTable.scala|  7 +++
 .../org/apache/spark/HeartbeatReceiverSuite.scala  |  2 +-
 .../scala/org/apache/spark/SparkThrowableSuite.scala   |  5 ++---
 .../spark/internal/plugin/PluginContainerSuite.scala   |  2 +-
 .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala  |  6 --
 .../apache/spark/deploy/yarn/ExecutorRunnable.scala|  2 +-
 .../org/apache/spark/deploy/yarn/YarnAllocator.scala   |  2 +-
 .../apache/spark/sql/catalyst/catalog/interface.scala  |  8 +++-
 .../catalyst/expressions/codegen/CodeGenerator.scala   |  2 +-
 .../catalyst/plans/logical/basicLogicalOperators.scala |  5 +++--
 .../scala/org/apache/spark/sql/DataFrameWriter.scala   |  2 +-
 .../apache/spark/sql/execution/command/tables.scala|  7 ---
 .../spark/sql/execution/datasources/DataSource.scala   |  4 ++--
 .../spark/sql/execution/datasources/FileFormat.scala   |  2 +-
 .../sql/execution/datasources/jdbc/JDBCOptions.scala   |  3 ++-
 .../sql/execution/datasources/v2/CacheTableExec.scala  |  3 ++-
 .../execution/datasources/v2/DataSourceV2Utils.scala   |  2 +-
 .../execution/datasources/v2/FileDataSourceV2.scala|  2 +-
 .../execution/datasources/v2/ShowCreateTableExec.scala | 18 ++
 .../execution/datasources/v2/V2SessionCatalog.scala|  4 ++--
 .../execution/streaming/state/RocksDBFileManager.scala |  6 +++---
 .../apache/spark/sql/execution/ui/ExecutionPage.scala  |  4 ++--
 .../apache/spark/sql/streaming/DataStreamReader.scala  |  2 +-
 .../apache/spark/sql/streaming/DataStreamWriter.scala  |  2 +-
 .../apache/spark/sql/hive/HiveExternalCatalog.scala| 11 ++-
 .../apache/spark/sql/hive/HiveMetastoreCatalog.scala   |  7 +++
 .../apache/spark/sql/hive/execution/HiveOptions.scala  |  6 +++---
 .../spark/sql/hive/HiveSchemaInferenceSuite.scala  |  2 +-
 .../org/apache/spark/sql/hive/StatisticsSuite.scala| 10 +-
 .../apache/spark/sql/hive/execution/HiveDDLSuite.scala |  8 
 .../hive/execution/command/ShowCreateTableSuite.scala  |  2 +-
 40 files changed, 93 insertions(+), 88 deletions(-)

diff --git 
a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala
 
b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala
index 026c4d560722..a86acd971a1c 100644
--- 
a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala
+++ 
b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala
@@ -102,7 +102,7 @@ class KafkaContinuousStream(
 }
 
 val startOffsets = newPartitionOffsets ++
-  oldStartPartitionOffsets.view.filterKeys(!deletedPartitions.contains(_))
+  oldStartPartitionOffsets.filter { case (k, _) => 
!deletedPartitions.contains(k) }
 knownPartitions = startOffsets.keySet
 
 startOffsets.toSeq.map {
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala 
b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 0fe72e28ea5b..2e1d7b9bc

(spark) branch master updated: [SPARK-32246][BUILD][INFRA] Enable `streaming-kinesis-asl` tests in Github Action CI

2023-12-04 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 64ec5f1017e1 [SPARK-32246][BUILD][INFRA] Enable 
`streaming-kinesis-asl` tests in Github Action CI
64ec5f1017e1 is described below

commit 64ec5f1017e1f2ca479060ca76f18b1c4a803b81
Author: Junyu Chen 
AuthorDate: Mon Dec 4 12:44:11 2023 -0800

[SPARK-32246][BUILD][INFRA] Enable `streaming-kinesis-asl` tests in Github 
Action CI

### What changes were proposed in this pull request?
This PR attempts to set up Kinesis tests in one of the existing Github 
Actions. Note that currently there are totally 57 tests in the Kinesis-asl 
module, and this PR enabled 35 of them. The remaining tests requires 
interaction with Amazon Kinesis service which would incur billing costs to 
users. Hence they are not included in the Github Action.

### Why are the changes needed?

Addressing the comments in this PR: 
https://github.com/apache/spark/pull/42581#issuecomment-1685925739 which 
attempts to upgrade the AWS SDK to v2 for Spark Kinesis connector. Since 
Kinesis tests are not being run in the Github Actions, there is no automated 
mechanism to verify the SDK v2 upgrade changes in this module.

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

No.

### How was this patch tested?

1. All existing Github Actions passed.
2. All Kinesis tests passed when running locally: `export 
ENABLE_KINESIS_TESTS=1 && mvn test -Pkinesis-asl -pl connector/kinesis-asl`
```
Tests: succeeded 57, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
[INFO] 

[INFO] BUILD SUCCESS
[INFO] 

[INFO] Total time:  13:25 min
[INFO] Finished at: 2023-11-12T00:15:49+08:00
[INFO] 

```

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

No.

Closes #43736 from junyuc25/junyuc25/kinesis-test.

Authored-by: Junyu Chen 
Signed-off-by: Dongjoon Hyun 
---
 .github/workflows/build_and_test.yml | 2 +-
 dev/sparktestsupport/modules.py  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/.github/workflows/build_and_test.yml 
b/.github/workflows/build_and_test.yml
index 4612b504ccdf..d58356ec1c5d 100644
--- a/.github/workflows/build_and_test.yml
+++ b/.github/workflows/build_and_test.yml
@@ -150,7 +150,7 @@ jobs:
   - >-
 mllib-local, mllib, graphx
   - >-
-streaming, sql-kafka-0-10, streaming-kafka-0-10,
+streaming, sql-kafka-0-10, streaming-kafka-0-10, 
streaming-kinesis-asl,
 yarn, kubernetes, hadoop-cloud, spark-ganglia-lgpl,
 connect, protobuf
 # Here, we split Hive and SQL tests into some of slow ones and the 
rest of them.
diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index 8995b7de0df9..15b2e8f186e5 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -359,7 +359,7 @@ streaming_kinesis_asl = Module(
 build_profile_flags=[
 "-Pkinesis-asl",
 ],
-environ={"ENABLE_KINESIS_TESTS": "1"},
+environ={"ENABLE_KINESIS_TESTS": "0"},
 sbt_test_goals=[
 "streaming-kinesis-asl/test",
 ],


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



(spark) branch branch-3.4 updated: [SPARK-46092][SQL][3.4] Don't push down Parquet row group filters that overflow

2023-12-04 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 05b5c9e2e3df [SPARK-46092][SQL][3.4] Don't push down Parquet row group 
filters that overflow
05b5c9e2e3df is described below

commit 05b5c9e2e3dfb7641e59895afc8ecb0f4f861127
Author: Johan Lasperas 
AuthorDate: Mon Dec 4 08:59:21 2023 -0800

[SPARK-46092][SQL][3.4] Don't push down Parquet row group filters that 
overflow

This is a cherry-pick from https://github.com/apache/spark/pull/44006 to 
spark 3.4

### What changes were proposed in this pull request?
This change adds a check for overflows when creating Parquet row group 
filters on an INT32 (byte/short/int) parquet type to avoid incorrectly skipping 
row groups if the predicate value doesn't fit in an INT. This can happen if the 
read schema is specified as LONG, e.g via `.schema("col LONG")`
While the Parquet readers don't support reading INT32 into a LONG, the 
overflow can lead to row groups being incorrectly skipped, bypassing the reader 
altogether and producing incorrect results instead of failing.

### Why are the changes needed?
Reading a parquet file containing INT32 values with a read schema specified 
as LONG can produce incorrect results today:
```
Seq(0).toDF("a").write.parquet(path)
spark.read.schema("a LONG").parquet(path).where(s"a < 
${Long.MaxValue}").collect()
```
will return an empty result. The correct result is either:
- Failing the query if the parquet reader doesn't support upcasting 
integers to longs (all parquet readers in Spark today)
- Return result `[0]` if the parquet reader supports that upcast (no 
readers in Spark as of now, but I'm looking into adding this capability).

### Does this PR introduce _any_ user-facing change?
The following:
```
Seq(0).toDF("a").write.parquet(path)
spark.read.schema("a LONG").parquet(path).where(s"a < 
${Long.MaxValue}").collect()
```
produces an (incorrect) empty result before this change. After this change, 
the read will fail, raising an error about the unsupported conversion from INT 
to LONG in the parquet reader.

### How was this patch tested?
- Added tests to `ParquetFilterSuite` to ensure that no row group filter is 
created when the predicate value overflows or when the value type isn't 
compatible with the parquet type
- Added test to `ParquetQuerySuite` covering the correctness issue 
described above.

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

Closes #44155 from johanl-db/SPARK-46092-row-group-skipping-overflow-3.4.

Authored-by: Johan Lasperas 
Signed-off-by: Dongjoon Hyun 
---
 .../datasources/parquet/ParquetFilters.scala   | 10 ++-
 .../datasources/parquet/ParquetFilterSuite.scala   | 71 ++
 .../datasources/parquet/ParquetQuerySuite.scala| 20 ++
 3 files changed, 99 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
index 6994e1ba39d9..5943dbdfb786 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.execution.datasources.parquet
 
-import java.lang.{Boolean => JBoolean, Double => JDouble, Float => JFloat, 
Long => JLong}
+import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float 
=> JFloat, Long => JLong, Short => JShort}
 import java.math.{BigDecimal => JBigDecimal}
 import java.nio.charset.StandardCharsets.UTF_8
 import java.sql.{Date, Timestamp}
@@ -612,7 +612,13 @@ class ParquetFilters(
 value == null || (nameToParquetField(name).fieldType match {
   case ParquetBooleanType => value.isInstanceOf[JBoolean]
   case ParquetIntegerType if value.isInstanceOf[Period] => true
-  case ParquetByteType | ParquetShortType | ParquetIntegerType => 
value.isInstanceOf[Number]
+  case ParquetByteType | ParquetShortType | ParquetIntegerType => value 
match {
+// Byte/Short/Int are all stored as INT32 in Parquet so filters are 
built using type Int.
+// We don't create a filter if the value would overflow.
+case _: JByte | _: JShort | _: Integer => true
+case v: JLong => v.longValue() >= Int.MinValue && v.longValue() <= 
Int.MaxValue
+case _ => false
+  }
   case ParquetLongType => 

(spark) branch branch-3.5 updated: [SPARK-46092][SQL][3.5] Don't push down Parquet row group filters that overflow

2023-12-04 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 97472c91ed56 [SPARK-46092][SQL][3.5] Don't push down Parquet row group 
filters that overflow
97472c91ed56 is described below

commit 97472c91ed5660c5af862e8da99d44a1c24f2815
Author: Johan Lasperas 
AuthorDate: Mon Dec 4 08:58:03 2023 -0800

[SPARK-46092][SQL][3.5] Don't push down Parquet row group filters that 
overflow

This is a cherry-pick from https://github.com/apache/spark/pull/44006 to 
spark 3.5

### What changes were proposed in this pull request?
This change adds a check for overflows when creating Parquet row group 
filters on an INT32 (byte/short/int) parquet type to avoid incorrectly skipping 
row groups if the predicate value doesn't fit in an INT. This can happen if the 
read schema is specified as LONG, e.g via `.schema("col LONG")`
While the Parquet readers don't support reading INT32 into a LONG, the 
overflow can lead to row groups being incorrectly skipped, bypassing the reader 
altogether and producing incorrect results instead of failing.

### Why are the changes needed?
Reading a parquet file containing INT32 values with a read schema specified 
as LONG can produce incorrect results today:
```
Seq(0).toDF("a").write.parquet(path)
spark.read.schema("a LONG").parquet(path).where(s"a < 
${Long.MaxValue}").collect()
```
will return an empty result. The correct result is either:
- Failing the query if the parquet reader doesn't support upcasting 
integers to longs (all parquet readers in Spark today)
- Return result `[0]` if the parquet reader supports that upcast (no 
readers in Spark as of now, but I'm looking into adding this capability).

### Does this PR introduce _any_ user-facing change?
The following:
```
Seq(0).toDF("a").write.parquet(path)
spark.read.schema("a LONG").parquet(path).where(s"a < 
${Long.MaxValue}").collect()
```
produces an (incorrect) empty result before this change. After this change, 
the read will fail, raising an error about the unsupported conversion from INT 
to LONG in the parquet reader.

### How was this patch tested?
- Added tests to `ParquetFilterSuite` to ensure that no row group filter is 
created when the predicate value overflows or when the value type isn't 
compatible with the parquet type
- Added test to `ParquetQuerySuite` covering the correctness issue 
described above.

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

Closes #44154 from johanl-db/SPARK-46092-row-group-skipping-overflow-3.5.

Authored-by: Johan Lasperas 
Signed-off-by: Dongjoon Hyun 
---
 .../datasources/parquet/ParquetFilters.scala   | 10 ++-
 .../datasources/parquet/ParquetFilterSuite.scala   | 71 ++
 .../datasources/parquet/ParquetQuerySuite.scala| 20 ++
 3 files changed, 99 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
index 5899b6621ad8..0983841dc8c2 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.execution.datasources.parquet
 
-import java.lang.{Boolean => JBoolean, Double => JDouble, Float => JFloat, 
Long => JLong}
+import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float 
=> JFloat, Long => JLong, Short => JShort}
 import java.math.{BigDecimal => JBigDecimal}
 import java.nio.charset.StandardCharsets.UTF_8
 import java.sql.{Date, Timestamp}
@@ -612,7 +612,13 @@ class ParquetFilters(
 value == null || (nameToParquetField(name).fieldType match {
   case ParquetBooleanType => value.isInstanceOf[JBoolean]
   case ParquetIntegerType if value.isInstanceOf[Period] => true
-  case ParquetByteType | ParquetShortType | ParquetIntegerType => 
value.isInstanceOf[Number]
+  case ParquetByteType | ParquetShortType | ParquetIntegerType => value 
match {
+// Byte/Short/Int are all stored as INT32 in Parquet so filters are 
built using type Int.
+// We don't create a filter if the value would overflow.
+case _: JByte | _: JShort | _: Integer => true
+case v: JLong => v.longValue() >= Int.MinValue && v.longValue() <= 
Int.MaxValue
+case _ => false
+  }
   case ParquetLongType => 

(spark) branch master updated: [SPARK-46231][PYTHON] Migrate all remaining `NotImplementedError` & `TypeError` into PySpark error framework

2023-12-04 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 9666bf37958e [SPARK-46231][PYTHON] Migrate all remaining 
`NotImplementedError` & `TypeError` into PySpark error framework
9666bf37958e is described below

commit 9666bf37958e5381278ca622bf7ec4b4ccb13d79
Author: Haejoon Lee 
AuthorDate: Mon Dec 4 08:54:50 2023 -0800

[SPARK-46231][PYTHON] Migrate all remaining `NotImplementedError` & 
`TypeError` into PySpark error framework

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

This PR proposes to migrate all remaining `NotImplementedError` and 
`TypeError`  from `pyspark/sql/*` into PySpark error framework, 
`PySparkNotImplementedError` with assigning dedicated error classes.

### Why are the changes needed?

To improve the error handling in PySpark.

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

No API changes, but the user-facing error messages will be improved.

### How was this patch tested?

The existing CI should pass.

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

No.

Closes #44148 from itholic/not_impl_and_type.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/sql/datasource.py| 21 +
 .../sql/tests/pandas/test_pandas_udf_grouped_agg.py |  3 +++
 python/pyspark/sql/udf.py   |  8 +++-
 3 files changed, 27 insertions(+), 5 deletions(-)

diff --git a/python/pyspark/sql/datasource.py b/python/pyspark/sql/datasource.py
index 1c5b6d663285..4713ca5366a7 100644
--- a/python/pyspark/sql/datasource.py
+++ b/python/pyspark/sql/datasource.py
@@ -19,6 +19,7 @@ from typing import final, Any, Dict, Iterator, List, 
Sequence, Tuple, Type, Unio
 
 from pyspark.sql import Row
 from pyspark.sql.types import StructType
+from pyspark.errors import PySparkNotImplementedError
 
 if TYPE_CHECKING:
 from pyspark.sql._typing import OptionalPrimitiveType
@@ -103,7 +104,10 @@ class DataSource(ABC):
 >>> def schema(self):
 ...   return StructType().add("a", "int").add("b", "string")
 """
-raise NotImplementedError
+raise PySparkNotImplementedError(
+error_class="NOT_IMPLEMENTED",
+message_parameters={"feature": "schema"},
+)
 
 def reader(self, schema: StructType) -> "DataSourceReader":
 """
@@ -121,7 +125,10 @@ class DataSource(ABC):
 reader : DataSourceReader
 A reader instance for this data source.
 """
-raise NotImplementedError
+raise PySparkNotImplementedError(
+error_class="NOT_IMPLEMENTED",
+message_parameters={"feature": "reader"},
+)
 
 def writer(self, schema: StructType, saveMode: str) -> "DataSourceWriter":
 """
@@ -142,7 +149,10 @@ class DataSource(ABC):
 writer : DataSourceWriter
 A writer instance for this data source.
 """
-raise NotImplementedError
+raise PySparkNotImplementedError(
+error_class="NOT_IMPLEMENTED",
+message_parameters={"feature": "writer"},
+)
 
 
 class InputPartition:
@@ -239,7 +249,10 @@ class DataSourceReader(ABC):
 >>> def partitions(self):
 ... return [RangeInputPartition(1, 3), RangeInputPartition(5, 10)]
 """
-raise NotImplementedError
+raise PySparkNotImplementedError(
+error_class="NOT_IMPLEMENTED",
+message_parameters={"feature": "partitions"},
+)
 
 @abstractmethod
 def read(self, partition: InputPartition) -> Iterator[Union[Tuple, Row]]:
diff --git a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py 
b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py
index b500be7a9695..455bb09a7dc4 100644
--- a/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py
+++ b/python/pyspark/sql/tests/pandas/test_pandas_udf_grouped_agg.py
@@ -720,6 +720,9 @@ class GroupedAggPandasUDFTestsMixin:
 
 
 class GroupedAggPandasUDFTests(GroupedAggPandasUDFTestsMixin, 
ReusedSQLTestCase):
+def test_unsupported_types(self):
+super().test_unsupported_types()
+
 pass
 
 
diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py
index 9ffdbb218711..351bcea3f389 100644
--- a/python/pyspark/sql/udf.py
+++ b/python/pyspark/sql/udf.py
@@ -339,7 +339,13 @@ class UserDefinedFunction:
 try:

(spark) branch master updated: [SPARK-46237][SQL][TESTS] Make `HiveDDLSuite` independently testable

2023-12-04 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 37d19b9ee0e4 [SPARK-46237][SQL][TESTS] Make `HiveDDLSuite` 
independently testable
37d19b9ee0e4 is described below

commit 37d19b9ee0e4e100e37358e71d771a2e42d01d88
Author: yangjie01 
AuthorDate: Mon Dec 4 08:52:23 2023 -0800

[SPARK-46237][SQL][TESTS] Make `HiveDDLSuite` independently testable

### What changes were proposed in this pull request?
When I test `HiveDDLSuite` with

```
build/sbt "hive/testOnly org.apache.spark.sql.hive.execution.HiveDDLSuite" 
-Phive
```
This test throws an error:

```
[info] - SPARK-34261: Avoid side effect if create exists temporary function 
*** FAILED *** (4 milliseconds)
[info]   java.util.NoSuchElementException: key not found: default
[info]   at scala.collection.MapOps.default(Map.scala:274)
[info]   at scala.collection.MapOps.default$(Map.scala:273)
[info]   at scala.collection.AbstractMap.default(Map.scala:405)
[info]   at scala.collection.MapOps.apply(Map.scala:176)
[info]   at scala.collection.MapOps.apply$(Map.scala:175)
[info]   at scala.collection.AbstractMap.apply(Map.scala:405)
[info]   at 
org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$445(HiveDDLSuite.scala:3275)
[info]   at 
org.apache.spark.sql.test.SQLTestUtilsBase.withUserDefinedFunction(SQLTestUtils.scala:256)
[info]   at 
org.apache.spark.sql.test.SQLTestUtilsBase.withUserDefinedFunction$(SQLTestUtils.scala:254)
[info]   at 
org.apache.spark.sql.execution.command.DDLSuite.withUserDefinedFunction(DDLSuite.scala:326)
[info]   at 
org.apache.spark.sql.hive.execution.HiveDDLSuite.$anonfun$new$444(HiveDDLSuite.scala:3267)
```

I manually printed the content of `spark.sparkContext.addedJars`, which is 
an empty `Map`.

However, when I execute

```
build/sbt "hive/testOnly org.apache.spark.sql.hive.execution.SQLQuerySuite 
org.apache.spark.sql.hive.execution.HiveDDLSuite" -Phive
```
All tests pass, and the content of `spark.sparkContext.addedJars` is

```
Map(default -> Map(spark://localhost:54875/jars/SPARK-21101-1.0.jar -> 
1701676986594, spark://localhost:54875/jars/hive-contrib-2.3.9.jar -> 
1701676944590, spark://localhost:54875/jars/TestUDTF.jar -> 1701676921340))
```

The reason why this failure is not reproduced in the GitHub Action test is 
because `SQLQuerySuite` is indeed executed before `HiveDDLSuite`.

So in the current PR, I change to use 
`.get("default").foreach(_.remove(k))` that the remove operation is only 
performed when `.get("default")` is not `None`.

### Why are the changes needed?
Make `HiveDDLSuite` independently testable.

### Does this PR introduce _any_ user-facing change?
No, just for test

### How was this patch tested?
- Pass Github Actions
- Manual check `HiveDDLSuite` with this pr and all test passed

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

Closes #44153 from LuciferYang/HiveDDLSuite.

    Authored-by: yangjie01 
Signed-off-by: Dongjoon Hyun 
---
 .../test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index c3a528da382a..2f5d1fcbb540 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -3270,7 +3270,7 @@ class HiveDDLSuite
   val jarName = "TestUDTF.jar"
   val jar = 
spark.asInstanceOf[TestHiveSparkSession].getHiveFile(jarName).toURI.toString
   spark.sparkContext.allAddedJars.keys.find(_.contains(jarName))
-.foreach(spark.sparkContext.addedJars("default").remove)
+.foreach(k => 
spark.sparkContext.addedJars.get("default").foreach(_.remove(k)))
   assert(!spark.sparkContext.listJars().exists(_.contains(jarName)))
   val e = intercept[AnalysisException] {
 sql("CREATE TEMPORARY FUNCTION f1 AS " +


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



(spark) branch master updated: [SPARK-46232][PYTHON] Migrate all remaining ValueError into PySpark error framework

2023-12-03 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 b23ae15da019 [SPARK-46232][PYTHON] Migrate all remaining ValueError 
into PySpark error framework
b23ae15da019 is described below

commit b23ae15da019082891d71853682329c2d24c2e9e
Author: Haejoon Lee 
AuthorDate: Sun Dec 3 22:49:30 2023 -0800

[SPARK-46232][PYTHON] Migrate all remaining ValueError into PySpark error 
framework

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

This PR proposes to migrate all remaining `ValueError`  from 
`pyspark/sql/*` into PySpark error framework, `PySparkValueError` with 
assigning dedicated error classes.

### Why are the changes needed?

To improve the error handling in PySpark.

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

No API changes, but the user-facing error messages will be improved.

### How was this patch tested?

The existing CI should pass.

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

No.

Closes #44149 from itholic/migrate_value_error.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/errors/error_classes.py   | 19 +--
 python/pyspark/sql/pandas/serializers.py |  5 +++--
 python/pyspark/sql/pandas/typehints.py   | 12 +---
 python/pyspark/sql/pandas/types.py   |  7 +--
 python/pyspark/sql/sql_formatter.py  |  7 ---
 5 files changed, 38 insertions(+), 12 deletions(-)

diff --git a/python/pyspark/errors/error_classes.py 
b/python/pyspark/errors/error_classes.py
index c7199ac938be..d0c0d1c115b0 100644
--- a/python/pyspark/errors/error_classes.py
+++ b/python/pyspark/errors/error_classes.py
@@ -287,6 +287,11 @@ ERROR_CLASSES_JSON = """
   "NumPy array input should be of  dimensions."
 ]
   },
+  "INVALID_NUMBER_OF_DATAFRAMES_IN_GROUP" : {
+"message" : [
+  "Invalid number of dataframes in group ."
+]
+  },
   "INVALID_PANDAS_UDF" : {
 "message" : [
   "Invalid function: "
@@ -803,9 +808,9 @@ ERROR_CLASSES_JSON = """
   "Expected  values for ``, got ."
 ]
   },
-  "TYPE_HINT_REQUIRED" : {
+  "TYPE_HINT_SHOULD_BE_SPECIFIED" : {
 "message" : [
-  "A  is required ."
+  "Type hints for  should be specified; however, got ."
 ]
   },
   "UDF_RETURN_TYPE" : {
@@ -888,6 +893,11 @@ ERROR_CLASSES_JSON = """
   "Unknown response: ."
 ]
   },
+  "UNKNOWN_VALUE_FOR" : {
+"message" : [
+  "Unknown value for ``."
+]
+  },
   "UNSUPPORTED_DATA_TYPE" : {
 "message" : [
   "Unsupported DataType ``."
@@ -983,6 +993,11 @@ ERROR_CLASSES_JSON = """
   "Value for `` only supports the 'pearson', got ''."
 ]
   },
+  "VALUE_NOT_PLAIN_COLUMN_REFERENCE" : {
+"message" : [
+  "Value  in  should be a plain column reference such as 
`df.col` or `col('column')`."
+]
+  },
   "VALUE_NOT_POSITIVE" : {
 "message" : [
   "Value for `` must be positive, got ''."
diff --git a/python/pyspark/sql/pandas/serializers.py 
b/python/pyspark/sql/pandas/serializers.py
index 8ffb7407714b..6c5bd826a023 100644
--- a/python/pyspark/sql/pandas/serializers.py
+++ b/python/pyspark/sql/pandas/serializers.py
@@ -707,8 +707,9 @@ class 
CogroupArrowUDFSerializer(ArrowStreamGroupUDFSerializer):
 yield batches1, batches2
 
 elif dataframes_in_group != 0:
-raise ValueError(
-"Invalid number of dataframes in group 
{0}".format(dataframes_in_group)
+raise PySparkValueError(
+error_class="INVALID_NUMBER_OF_DATAFRAMES_IN_GROUP",
+message_parameters={"dataframes_in_group": 
str(dataframes_in_group)},
 )
 
 
diff --git a/python/pyspark/sql/pandas/typehints.py 
b/python/pyspark/sql/pandas/typehints.py
index f0c13e66a63d..37ba02a94d58 100644
--- a/python/pyspark/sql/pandas/typehints.py
+++ b/python/pyspark/sql/pandas/typehints.py
@@ -18,7 +18,7 @@ from inspect import Signature
 from typing import Any, Callable, Dict, Optional, Union, TYPE_CHECKING
 
 from pyspark.sql.pandas.utils import require_minimum_pandas_version
-from pyspark.errors import PySparkNotImplementedError
+from pyspark.errors import PySparkNotImplementedError, PySparkValueError
 
 if TYPE_CHECKING:
 from pyspark.sql.pandas._typing import (
@@ -51,12 +51,18 @@ def infer_eval_type(
 annotati

(spark) branch branch-3.4 updated: [SPARK-46182][CORE] Track `lastTaskFinishTime` using the exact task finished event

2023-12-03 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new b8750d5c0b41 [SPARK-46182][CORE] Track `lastTaskFinishTime` using the 
exact task finished event
b8750d5c0b41 is described below

commit b8750d5c0b416137ce802cf73dd92b0fc7ff5467
Author: Xingbo Jiang 
AuthorDate: Sun Dec 3 22:08:20 2023 -0800

[SPARK-46182][CORE] Track `lastTaskFinishTime` using the exact task 
finished event

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

We found a race condition between lastTaskRunningTime and 
lastShuffleMigrationTime that could lead to a decommissioned executor exit 
before all the shuffle blocks have been discovered. The issue could lead to 
immediate task retry right after an executor exit, thus longer query execution 
time.

To fix the issue, we choose to update the lastTaskRunningTime only when a 
task updates its status to finished through the StatusUpdate event. This is 
better than the current approach (which use a thread to check for number of 
running tasks every second), because in this way we clearly know whether the 
shuffle block refresh happened after all tasks finished running or not, thus 
resolved the race condition mentioned above.

### Why are the changes needed?

To fix a race condition that could lead to shuffle data lost, thus longer 
query execution time.

### How was this patch tested?

This is a very subtle race condition that is hard to write a unit test 
using current unit test framework. And we are confident the change is low risk. 
Thus only verify by passing all the existing tests.

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

No

Closes #44090 from jiangxb1987/SPARK-46182.

Authored-by: Xingbo Jiang 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 6f112f7b1a50a2b8a59952c69f67dd5f80ab6633)
Signed-off-by: Dongjoon Hyun 
---
 .../spark/executor/CoarseGrainedExecutorBackend.scala| 16 +++-
 1 file changed, 7 insertions(+), 9 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index c695a9ec2851..537522326fc7 100644
--- 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -21,7 +21,7 @@ import java.net.URL
 import java.nio.ByteBuffer
 import java.util.Locale
 import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicBoolean
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
 
 import scala.util.{Failure, Success}
 import scala.util.control.NonFatal
@@ -80,6 +80,10 @@ private[spark] class CoarseGrainedExecutorBackend(
 
   private var decommissioned = false
 
+  // Track the last time in ns that at least one task is running. If no task 
is running and all
+  // shuffle/RDD data migration are done, the decommissioned executor should 
exit.
+  private var lastTaskFinishTime = new AtomicLong(System.nanoTime())
+
   override def onStart(): Unit = {
 if (env.conf.get(DECOMMISSION_ENABLED)) {
   val signal = env.conf.get(EXECUTOR_DECOMMISSION_SIGNAL)
@@ -269,6 +273,7 @@ private[spark] class CoarseGrainedExecutorBackend(
 val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources)
 if (TaskState.isFinished(state)) {
   taskResources.remove(taskId)
+  lastTaskFinishTime.set(System.nanoTime())
 }
 driver match {
   case Some(driverRef) => driverRef.send(msg)
@@ -341,7 +346,6 @@ private[spark] class CoarseGrainedExecutorBackend(
 
   val shutdownThread = new Thread("wait-for-blocks-to-migrate") {
 override def run(): Unit = {
-  var lastTaskRunningTime = System.nanoTime()
   val sleep_time = 1000 // 1s
   // This config is internal and only used by unit tests to force an 
executor
   // to hang around for longer when decommissioned.
@@ -358,7 +362,7 @@ private[spark] class CoarseGrainedExecutorBackend(
 val (migrationTime, allBlocksMigrated) = 
env.blockManager.lastMigrationInfo()
 // We can only trust allBlocksMigrated boolean value if there 
were no tasks running
 // since the start of computing it.
-if (allBlocksMigrated && (migrationTime > 
lastTaskRunningTime)) {
+if (allBlocksMigrated && (migrationTime > 
lastTaskFinishTime.get())) {
   logInfo("No running tasks, all blocks migrated, stopping.")
   exitExecutor(0, ExecutorLossMessage.decommissionFinished, 
notifyDriver = true)
 

(spark) branch branch-3.5 updated: [SPARK-46182][CORE] Track `lastTaskFinishTime` using the exact task finished event

2023-12-03 Thread dongjoon
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
 new 273ef5708fc3 [SPARK-46182][CORE] Track `lastTaskFinishTime` using the 
exact task finished event
273ef5708fc3 is described below

commit 273ef5708fc33872cfe3091627617bbac8fdd56f
Author: Xingbo Jiang 
AuthorDate: Sun Dec 3 22:08:20 2023 -0800

[SPARK-46182][CORE] Track `lastTaskFinishTime` using the exact task 
finished event

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

We found a race condition between lastTaskRunningTime and 
lastShuffleMigrationTime that could lead to a decommissioned executor exit 
before all the shuffle blocks have been discovered. The issue could lead to 
immediate task retry right after an executor exit, thus longer query execution 
time.

To fix the issue, we choose to update the lastTaskRunningTime only when a 
task updates its status to finished through the StatusUpdate event. This is 
better than the current approach (which use a thread to check for number of 
running tasks every second), because in this way we clearly know whether the 
shuffle block refresh happened after all tasks finished running or not, thus 
resolved the race condition mentioned above.

### Why are the changes needed?

To fix a race condition that could lead to shuffle data lost, thus longer 
query execution time.

### How was this patch tested?

This is a very subtle race condition that is hard to write a unit test 
using current unit test framework. And we are confident the change is low risk. 
Thus only verify by passing all the existing tests.

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

No

Closes #44090 from jiangxb1987/SPARK-46182.

Authored-by: Xingbo Jiang 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit 6f112f7b1a50a2b8a59952c69f67dd5f80ab6633)
Signed-off-by: Dongjoon Hyun 
---
 .../spark/executor/CoarseGrainedExecutorBackend.scala| 16 +++-
 1 file changed, 7 insertions(+), 9 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index c695a9ec2851..537522326fc7 100644
--- 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -21,7 +21,7 @@ import java.net.URL
 import java.nio.ByteBuffer
 import java.util.Locale
 import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicBoolean
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
 
 import scala.util.{Failure, Success}
 import scala.util.control.NonFatal
@@ -80,6 +80,10 @@ private[spark] class CoarseGrainedExecutorBackend(
 
   private var decommissioned = false
 
+  // Track the last time in ns that at least one task is running. If no task 
is running and all
+  // shuffle/RDD data migration are done, the decommissioned executor should 
exit.
+  private var lastTaskFinishTime = new AtomicLong(System.nanoTime())
+
   override def onStart(): Unit = {
 if (env.conf.get(DECOMMISSION_ENABLED)) {
   val signal = env.conf.get(EXECUTOR_DECOMMISSION_SIGNAL)
@@ -269,6 +273,7 @@ private[spark] class CoarseGrainedExecutorBackend(
 val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources)
 if (TaskState.isFinished(state)) {
   taskResources.remove(taskId)
+  lastTaskFinishTime.set(System.nanoTime())
 }
 driver match {
   case Some(driverRef) => driverRef.send(msg)
@@ -341,7 +346,6 @@ private[spark] class CoarseGrainedExecutorBackend(
 
   val shutdownThread = new Thread("wait-for-blocks-to-migrate") {
 override def run(): Unit = {
-  var lastTaskRunningTime = System.nanoTime()
   val sleep_time = 1000 // 1s
   // This config is internal and only used by unit tests to force an 
executor
   // to hang around for longer when decommissioned.
@@ -358,7 +362,7 @@ private[spark] class CoarseGrainedExecutorBackend(
 val (migrationTime, allBlocksMigrated) = 
env.blockManager.lastMigrationInfo()
 // We can only trust allBlocksMigrated boolean value if there 
were no tasks running
 // since the start of computing it.
-if (allBlocksMigrated && (migrationTime > 
lastTaskRunningTime)) {
+if (allBlocksMigrated && (migrationTime > 
lastTaskFinishTime.get())) {
   logInfo("No running tasks, all blocks migrated, stopping.")
   exitExecutor(0, ExecutorLossMessage.decommissionFinished, 
notifyDriver = true)
 

(spark) branch master updated: [SPARK-46182][CORE] Track `lastTaskFinishTime` using the exact task finished event

2023-12-03 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 6f112f7b1a50 [SPARK-46182][CORE] Track `lastTaskFinishTime` using the 
exact task finished event
6f112f7b1a50 is described below

commit 6f112f7b1a50a2b8a59952c69f67dd5f80ab6633
Author: Xingbo Jiang 
AuthorDate: Sun Dec 3 22:08:20 2023 -0800

[SPARK-46182][CORE] Track `lastTaskFinishTime` using the exact task 
finished event

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

We found a race condition between lastTaskRunningTime and 
lastShuffleMigrationTime that could lead to a decommissioned executor exit 
before all the shuffle blocks have been discovered. The issue could lead to 
immediate task retry right after an executor exit, thus longer query execution 
time.

To fix the issue, we choose to update the lastTaskRunningTime only when a 
task updates its status to finished through the StatusUpdate event. This is 
better than the current approach (which use a thread to check for number of 
running tasks every second), because in this way we clearly know whether the 
shuffle block refresh happened after all tasks finished running or not, thus 
resolved the race condition mentioned above.

### Why are the changes needed?

To fix a race condition that could lead to shuffle data lost, thus longer 
query execution time.

### How was this patch tested?

This is a very subtle race condition that is hard to write a unit test 
using current unit test framework. And we are confident the change is low risk. 
Thus only verify by passing all the existing tests.

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

No

Closes #44090 from jiangxb1987/SPARK-46182.

Authored-by: Xingbo Jiang 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/executor/CoarseGrainedExecutorBackend.scala| 16 +++-
 1 file changed, 7 insertions(+), 9 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index f1a9aa353e76..4bf4929c1339 100644
--- 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -21,7 +21,7 @@ import java.net.URL
 import java.nio.ByteBuffer
 import java.util.Locale
 import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicBoolean
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
 
 import scala.util.{Failure, Success}
 import scala.util.control.NonFatal
@@ -77,6 +77,10 @@ private[spark] class CoarseGrainedExecutorBackend(
 
   private var decommissioned = false
 
+  // Track the last time in ns that at least one task is running. If no task 
is running and all
+  // shuffle/RDD data migration are done, the decommissioned executor should 
exit.
+  private var lastTaskFinishTime = new AtomicLong(System.nanoTime())
+
   override def onStart(): Unit = {
 if (env.conf.get(DECOMMISSION_ENABLED)) {
   val signal = env.conf.get(EXECUTOR_DECOMMISSION_SIGNAL)
@@ -273,6 +277,7 @@ private[spark] class CoarseGrainedExecutorBackend(
 val msg = StatusUpdate(executorId, taskId, state, data, cpus, resources)
 if (TaskState.isFinished(state)) {
   taskResources.remove(taskId)
+  lastTaskFinishTime.set(System.nanoTime())
 }
 driver match {
   case Some(driverRef) => driverRef.send(msg)
@@ -345,7 +350,6 @@ private[spark] class CoarseGrainedExecutorBackend(
 
   val shutdownThread = new Thread("wait-for-blocks-to-migrate") {
 override def run(): Unit = {
-  var lastTaskRunningTime = System.nanoTime()
   val sleep_time = 1000 // 1s
   // This config is internal and only used by unit tests to force an 
executor
   // to hang around for longer when decommissioned.
@@ -362,7 +366,7 @@ private[spark] class CoarseGrainedExecutorBackend(
 val (migrationTime, allBlocksMigrated) = 
env.blockManager.lastMigrationInfo()
 // We can only trust allBlocksMigrated boolean value if there 
were no tasks running
 // since the start of computing it.
-if (allBlocksMigrated && (migrationTime > 
lastTaskRunningTime)) {
+if (allBlocksMigrated && (migrationTime > 
lastTaskFinishTime.get())) {
   logInfo("No running tasks, all blocks migrated, stopping.")
   exitExecutor(0, ExecutorLossMessage.decommissionFinished, 
notifyDriver = true)
 } else {
@@ -374,12 +378,6 @@ private[spark] class CoarseGrainedExecutorBackend(
   }
 

(spark) branch master updated (0c029e70706c -> 712352e37ec5)

2023-12-03 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 0c029e70706c [SPARK-46218][BUILD] Upgrade commons-cli to 1.6.0
 add 712352e37ec5 [SPARK-40559][PYTHON][DOCS][FOLLOW-UP] Fix the docstring 
and document both applyInArrows

No new revisions were added by this update.

Summary of changes:
 python/docs/source/reference/pyspark.sql/grouping.rst |  2 ++
 python/pyspark/sql/pandas/group_ops.py| 15 +++
 2 files changed, 9 insertions(+), 8 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-46218][BUILD] Upgrade commons-cli to 1.6.0

2023-12-03 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 0c029e70706c [SPARK-46218][BUILD] Upgrade commons-cli to 1.6.0
0c029e70706c is described below

commit 0c029e70706c7e1a4c3a7bb763dbbcb4fe1ccd9f
Author: panbingkun 
AuthorDate: Sun Dec 3 21:27:21 2023 -0800

[SPARK-46218][BUILD] Upgrade commons-cli to 1.6.0

### What changes were proposed in this pull request?
The pr aims to upgrade `commons-cli` from `1.5.0` to `1.6.0`.

### Why are the changes needed?
- The last upgrade occurred two years ago, 
https://github.com/apache/spark/pull/34707
- The full release notes: 
https://commons.apache.org/proper/commons-cli/changes-report.html#a1.6.0
- The version mainly focus on fixing bugs:
Fix NPE in CommandLine.resolveOption(String). Fixes 
[CLI-283](https://issues.apache.org/jira/browse/CLI-283).
CommandLine.addOption(Option) should not allow a null Option. Fixes 
[CLI-283](https://issues.apache.org/jira/browse/CLI-283).
CommandLine.addArgs(String) should not allow a null String. Fixes 
[CLI-283](https://issues.apache.org/jira/browse/CLI-283).
NullPointerException thrown by CommandLineParser.parse(). Fixes 
[CLI-317](https://issues.apache.org/jira/browse/CLI-317).
StringIndexOutOfBoundsException thrown by CommandLineParser.parse(). Fixes 
[CLI-313](https://issues.apache.org/jira/browse/CLI-313).

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

### How was this patch tested?
Pass GA.

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

Closes #44132 from panbingkun/SPARK-46218.

Authored-by: panbingkun 
Signed-off-by: Dongjoon Hyun 
---
 dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +-
 pom.xml   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 
b/dev/deps/spark-deps-hadoop-3-hive-2.3
index f1d675d92b6d..ebfe6acad960 100644
--- a/dev/deps/spark-deps-hadoop-3-hive-2.3
+++ b/dev/deps/spark-deps-hadoop-3-hive-2.3
@@ -35,7 +35,7 @@ breeze_2.13/2.1.0//breeze_2.13-2.1.0.jar
 cats-kernel_2.13/2.8.0//cats-kernel_2.13-2.8.0.jar
 chill-java/0.10.0//chill-java-0.10.0.jar
 chill_2.13/0.10.0//chill_2.13-0.10.0.jar
-commons-cli/1.5.0//commons-cli-1.5.0.jar
+commons-cli/1.6.0//commons-cli-1.6.0.jar
 commons-codec/1.16.0//commons-codec-1.16.0.jar
 commons-collections/3.2.2//commons-collections-3.2.2.jar
 commons-collections4/4.4//commons-collections4-4.4.jar
diff --git a/pom.xml b/pom.xml
index 2a259cfd322b..27ee42f103dd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -220,7 +220,7 @@
 2.70.0
 3.1.0
 1.1.0
-1.5.0
+1.6.0
 1.70
 1.9.0
 4.1.100.Final


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



(spark) branch master updated: [SPARK-46227][SQL] Move `withSQLConf` from `SQLHelper` to `SQLConfHelper`

2023-12-03 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 aee6b1582775 [SPARK-46227][SQL] Move `withSQLConf` from `SQLHelper` to 
`SQLConfHelper`
aee6b1582775 is described below

commit aee6b158277537709a717223b518923431bca0a6
Author: ulysses-you 
AuthorDate: Sun Dec 3 21:23:34 2023 -0800

[SPARK-46227][SQL] Move `withSQLConf` from `SQLHelper` to `SQLConfHelper`

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

This pr moves method `withSQLConf` from `SQLHelper` in catalyst test module 
to `SQLConfHelper` trait in catalyst module. To make it easy to use such case: 
`val x = withSQLConf {}`, this pr also changes its return type.

### Why are the changes needed?

A part of https://github.com/apache/spark/pull/44013

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

no

### How was this patch tested?

Pass CI

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

no

Closes #44142 from ulysses-you/withSQLConf.

Authored-by: ulysses-you 
Signed-off-by: Dongjoon Hyun 
---
 .../apache/spark/sql/catalyst/SQLConfHelper.scala  | 29 
 .../spark/sql/catalyst/plans/SQLHelper.scala   | 32 ++
 .../sql/internal/ExecutorSideSQLConfSuite.scala|  2 +-
 .../org/apache/spark/sql/test/SQLTestUtils.scala   |  2 +-
 .../spark/sql/hive/execution/HiveSerDeSuite.scala  |  2 +-
 5 files changed, 34 insertions(+), 33 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SQLConfHelper.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SQLConfHelper.scala
index cee35cdb8d84..f4605b9218f0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SQLConfHelper.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SQLConfHelper.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.catalyst
 
+import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.internal.SQLConf
 
 /**
@@ -29,4 +30,32 @@ trait SQLConfHelper {
* See [[SQLConf.get]] for more information.
*/
   def conf: SQLConf = SQLConf.get
+
+  /**
+   * Sets all SQL configurations specified in `pairs`, calls `f`, and then 
restores all SQL
+   * configurations.
+   */
+  protected def withSQLConf[T](pairs: (String, String)*)(f: => T): T = {
+val conf = SQLConf.get
+val (keys, values) = pairs.unzip
+val currentValues = keys.map { key =>
+  if (conf.contains(key)) {
+Some(conf.getConfString(key))
+  } else {
+None
+  }
+}
+keys.lazyZip(values).foreach { (k, v) =>
+  if (SQLConf.isStaticConfigKey(k)) {
+throw new AnalysisException(s"Cannot modify the value of a static 
config: $k")
+  }
+  conf.setConfString(k, v)
+}
+try f finally {
+  keys.zip(currentValues).foreach {
+case (key, Some(value)) => conf.setConfString(key, value)
+case (key, None) => conf.unsetConf(key)
+  }
+}
+  }
 }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala
index eb844e6f057f..92681613bd83 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala
@@ -23,41 +23,13 @@ import scala.util.control.NonFatal
 
 import org.scalatest.Assertions.fail
 
-import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.SQLConfHelper
 import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
 import org.apache.spark.sql.catalyst.util.DateTimeUtils.getZoneId
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.util.Utils
 
-trait SQLHelper {
-
-  /**
-   * Sets all SQL configurations specified in `pairs`, calls `f`, and then 
restores all SQL
-   * configurations.
-   */
-  protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
-val conf = SQLConf.get
-val (keys, values) = pairs.unzip
-val currentValues = keys.map { key =>
-  if (conf.contains(key)) {
-Some(conf.getConfString(key))
-  } else {
-None
-  }
-}
-keys.lazyZip(values).foreach { (k, v) =>
-  if (SQLConf.isStaticConfigKey(k)) {
-throw new AnalysisException(s"Cannot modify the value of a static 
config: $k")
-  }
-  conf.setConfString(k, v)
-}
-try f finally {
-  keys.zip(currentValues).foreach {
-case (key, Some(value)) => conf.setConfString(key, value)
-case (key, None) => conf.unsetConf(key)
-  }
-}
-  }
+trait SQLHelper extends SQLConf

(spark) branch master updated: [SPARK-46217][CORE][TESTS] Include `Driver/App` data in `PersistenceEngineBenchmark`

2023-12-03 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 5031e52f9e0 [SPARK-46217][CORE][TESTS] Include `Driver/App` data in 
`PersistenceEngineBenchmark`
5031e52f9e0 is described below

commit 5031e52f9e032e8e450af9fcd294f5b53e2c4cfd
Author: Dongjoon Hyun 
AuthorDate: Sun Dec 3 15:50:09 2023 -0800

[SPARK-46217][CORE][TESTS] Include `Driver/App` data in 
`PersistenceEngineBenchmark`

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

This PR aims to include `DirverInfo` and `ApplicationInfo` data in 
`PersistenceEngineBenchmark`.

### Why are the changes needed?

Previously, `PersistenceEngine` recovers three kind of information. 
Previously, `PersistenceEngineBenchmark ` focused on `WorkerInfo` only. This PR 
will add two other informations to be more complete.


https://github.com/apache/spark/blob/3da2e5c632468ec7cf7001255c1a44197b46ce30/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala#L56-L78

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

No. This is a test improvement.

### How was this patch tested?

Manual tests.

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

No.

Closes #44130 from dongjoon-hyun/SPARK-46217.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../PersistenceEngineBenchmark-jdk21-results.txt   | 28 +-
 .../PersistenceEngineBenchmark-results.txt | 28 +-
 .../deploy/master/PersistenceEngineBenchmark.scala | 65 +-
 3 files changed, 80 insertions(+), 41 deletions(-)

diff --git a/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt 
b/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
index 38e74ed6b53..314fb6958b6 100644
--- a/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
+++ b/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
@@ -4,20 +4,20 @@ PersistenceEngineBenchmark
 
 OpenJDK 64-Bit Server VM 21.0.1+12-LTS on Linux 5.15.0-1051-azure
 AMD EPYC 7763 64-Core Processor
-2000 Workers: Best Time(ms)   Avg 
Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   Relative
+1000 Workers: Best Time(ms)   Avg 
Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   Relative
 

-ZooKeeperPersistenceEngine with JavaSerializer 2254
   2329 119  0.0 1126867.1   1.0X
-ZooKeeperPersistenceEngine with KryoSerializer 1911
   1912   1  0.0  955667.1   1.2X
-FileSystemPersistenceEngine with JavaSerializer 438
448  15  0.0  218868.1   5.1X
-FileSystemPersistenceEngine with JavaSerializer (lz4)   187
195   8  0.0   93337.8  12.1X
-FileSystemPersistenceEngine with JavaSerializer (lzf)   193
216  20  0.0   96678.8  11.7X
-FileSystemPersistenceEngine with JavaSerializer (snappy)175
183  10  0.0   87652.3  12.9X
-FileSystemPersistenceEngine with JavaSerializer (zstd)  243
255  14  0.0  121695.2   9.3X
-FileSystemPersistenceEngine with KryoSerializer 150
160  15  0.0   75089.7  15.0X
-FileSystemPersistenceEngine with KryoSerializer (lz4)   170
177  10  0.0   84996.7  13.3X
-FileSystemPersistenceEngine with KryoSerializer (lzf)   192
203  12  0.0   96019.1  11.7X
-FileSystemPersistenceEngine with KryoSerializer (snappy)184
202  16  0.0   92241.3  12.2X
-FileSystemPersistenceEngine with KryoSerializer (zstd)  232
238   5  0.0  116075.2   9.7X
-BlackHolePersistenceEngine0
  0   0 27.3  36.6   30761.0X
+ZooKeeperPersistenceEngine with JavaSerializer 5402
   5546 233  0.0 5402030.8   1.0X
+ZooKeeperPersistenceEngine with KryoSerializer 4185
   4220  32  0.0 4184623.1   1.3X
+FileSystemPersistenceEngine with JavaSerializer1591
   1634  37  0.0 1590836.4   3.4X
+FileSystemPersistenceEngine with JavaSerializer (lz4

(spark) branch master updated: [SPARK-46216][CORE] Improve `FileSystemPersistenceEngine` to support compressions

2023-12-03 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 3da2e5c6324 [SPARK-46216][CORE] Improve `FileSystemPersistenceEngine` 
to support compressions
3da2e5c6324 is described below

commit 3da2e5c632468ec7cf7001255c1a44197b46ce30
Author: Dongjoon Hyun 
AuthorDate: Sun Dec 3 00:26:16 2023 -0800

[SPARK-46216][CORE] Improve `FileSystemPersistenceEngine` to support 
compressions

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

This PR aims to improve `FileSystemPersistenceEngine` to support 
compressions via a new configuration, `spark.deploy.recoveryCompressionCodec`.

### Why are the changes needed?

To allow the users to choose a proper compression codec for their 
workloads. For `JavaSerializer` case, `LZ4` compression is **2x** faster than 
the baseline (no compression).
```
OpenJDK 64-Bit Server VM 17.0.9+9-LTS on Linux 5.15.0-1051-azure
AMD EPYC 7763 64-Core Processor
2000 Workers: Best Time(ms)   
Avg Time(ms)   Stdev(ms)Rate(M/s)   Per Row(ns)   Relative


ZooKeeperPersistenceEngine with JavaSerializer 2276 
  2360 115  0.0 1137909.6   1.0X
ZooKeeperPersistenceEngine with KryoSerializer 1883 
  1906  34  0.0  941364.2   1.2X
FileSystemPersistenceEngine with JavaSerializer 431 
   436   7  0.0  215436.9   5.3X
FileSystemPersistenceEngine with JavaSerializer (lz4)   209 
   216   9  0.0  104404.1  10.9X
FileSystemPersistenceEngine with JavaSerializer (lzf)   199 
   202   2  0.0   99489.5  11.4X
FileSystemPersistenceEngine with JavaSerializer (snappy)192 
   199   9  0.0   95872.9  11.9X
FileSystemPersistenceEngine with JavaSerializer (zstd)  258 
   264   6  0.0  129249.4   8.8X
FileSystemPersistenceEngine with KryoSerializer 139 
   151  13  0.0   69374.5  16.4X
FileSystemPersistenceEngine with KryoSerializer (lz4)   159 
   165   8  0.0   79588.9  14.3X
FileSystemPersistenceEngine with KryoSerializer (lzf)   180 
   195  18  0.0   89844.0  12.7X
FileSystemPersistenceEngine with KryoSerializer (snappy)164 
   183  18  0.0   82016.0  13.9X
FileSystemPersistenceEngine with KryoSerializer (zstd)  206 
   218  11  0.0  102838.9  11.1X
BlackHolePersistenceEngine0 
 0   0 35.1  28.5   39908.5X
```

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

No, this is a new feature.

### How was this patch tested?

Pass the CIs with the newly added test case.

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

No.

Closes #44129 from dongjoon-hyun/SPARK-46216.

Authored-by: Dongjoon Hyun 
Signed-off-by: Dongjoon Hyun 
---
 .../PersistenceEngineBenchmark-jdk21-results.txt   | 22 
 .../PersistenceEngineBenchmark-results.txt | 22 
 .../master/FileSystemPersistenceEngine.scala   | 10 --
 .../spark/deploy/master/RecoveryModeFactory.scala  |  6 ++--
 .../org/apache/spark/internal/config/Deploy.scala  |  7 
 .../apache/spark/deploy/master/MasterSuite.scala   | 40 ++
 .../deploy/master/PersistenceEngineBenchmark.scala | 19 --
 .../deploy/master/PersistenceEngineSuite.scala | 13 +++
 8 files changed, 118 insertions(+), 21 deletions(-)

diff --git a/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt 
b/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
index 65dbfd0990d..38e74ed6b53 100644
--- a/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
+++ b/core/benchmarks/PersistenceEngineBenchmark-jdk21-results.txt
@@ -4,12 +4,20 @@ PersistenceEngineBenchmark
 
 OpenJDK 64-Bit Server VM 21.0.1+12-LTS on Linux 5.15.0-1051-azure
 AMD EPYC 7763 64-Core Processor
-1000 Workers:Best Time(ms)   Avg Time(ms)  
 Stdev(ms)Rate(M/s)   Per Row(ns)   Relative

(spark) branch master updated: [SPARK-46206][PS] Use a narrower scope exception for SQL processor

2023-12-02 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 73a09ed7bd7 [SPARK-46206][PS] Use a narrower scope exception for SQL 
processor
73a09ed7bd7 is described below

commit 73a09ed7bd7372779e25d65498c4ab6b8496f0a8
Author: Haejoon Lee 
AuthorDate: Sat Dec 2 21:42:09 2023 -0800

[SPARK-46206][PS] Use a narrower scope exception for SQL processor

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

This PR proposes to refine the exception handling in SQL processor 
functions by replacing the general `Exception` class with more specific 
exception types.

### Why are the changes needed?

The current exception handling uses the broad `Exception` type, which can 
obscure the root cause of issues. By specifying more accurate exceptions, the 
code becomes clearer:

- In `_get_local_scope()`, an `IndexError` is more appropriate as it 
explicitly handles the case where the index is out of range when accessing the 
call stack using `inspect.stack()`.
- In `_get_ipython_scope()`, `AttributeError` and `ModuleNotFoundError` 
could occur if the IPython environment is not available or the expected 
attributes in the IPython shell object are missing.

Using these specific exceptions enhances the maintainability and 
readability of the code, making it easier for developers to understand and 
handle errors more effectively.

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

### How was this patch tested?

The existing test suite `pyspark.pandas.tests.test_sql::SQLTests` should 
pass.

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

No.

Closes #44114 from itholic/refine_sql_error.

Authored-by: Haejoon Lee 
Signed-off-by: Dongjoon Hyun 
---
 python/pyspark/pandas/sql_processor.py | 8 ++--
 1 file changed, 2 insertions(+), 6 deletions(-)

diff --git a/python/pyspark/pandas/sql_processor.py 
b/python/pyspark/pandas/sql_processor.py
index 1bd1cb9823c..b047417b763 100644
--- a/python/pyspark/pandas/sql_processor.py
+++ b/python/pyspark/pandas/sql_processor.py
@@ -206,9 +206,7 @@ def _get_local_scope() -> Dict[str, Any]:
 # Get 2 scopes above (_get_local_scope -> sql -> ...) to capture the vars 
there.
 try:
 return inspect.stack()[_CAPTURE_SCOPES][0].f_locals
-except Exception:
-# TODO (rxin, thunterdb): use a narrower scope exception.
-# See https://github.com/databricks/koalas/pull/448
+except IndexError:
 return {}
 
 
@@ -222,9 +220,7 @@ def _get_ipython_scope() -> Dict[str, Any]:
 
 shell = get_ipython()
 return shell.user_ns
-except Exception:
-# TODO (rxin, thunterdb): use a narrower scope exception.
-# See https://github.com/databricks/koalas/pull/448
+except (AttributeError, ModuleNotFoundError):
 return None
 
 


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



(spark) branch master updated: [SPARK-46212][CORE][SQL][SS][CONNECT][MLLIB][GRAPHX][DSTREAM][PROTOBUF][EXAMPLES] Use other functions to simplify the code pattern of `s.c.MapOps#view.mapValues`

2023-12-02 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 938b7f58051 
[SPARK-46212][CORE][SQL][SS][CONNECT][MLLIB][GRAPHX][DSTREAM][PROTOBUF][EXAMPLES]
 Use other functions to simplify the code pattern of `s.c.MapOps#view.mapValues`
938b7f58051 is described below

commit 938b7f580519e3da64004185f7083ae63cf99bc0
Author: yangjie01 
AuthorDate: Sat Dec 2 21:39:14 2023 -0800


[SPARK-46212][CORE][SQL][SS][CONNECT][MLLIB][GRAPHX][DSTREAM][PROTOBUF][EXAMPLES]
 Use other functions to simplify the code pattern of `s.c.MapOps#view.mapValues`

### What changes were proposed in this pull request?
This pr simplifies `s.c.MapOps.view.mapValues` using the following approach:

- For the `s.c.immutable.MapOps` type, replace it with the 
`s.c.immutable.MapOps#transform` function.

```scala
def transform[W](f: (K, V) => W): CC[K, W] = map { case (k, v) => (k, f(k, 
v)) }
```

Like the case in `CountMinSketchSuite`:


https://github.com/apache/spark/blob/0d40b1aea758b95a4416c8653599af8713a4aa16/common/sketch/src/test/scala/org/apache/spark/util/sketch/CountMinSketchSuite.scala#L59

- For the `s.c.MapOps` type, since the `transform` function does not exist 
for this type, replace it directly with the `map` function.

```scala
def map[K2, V2](f: ((K, V)) => (K2, V2)): CC[K2, V2] = mapFactory.from(new 
View.Map(this, f))
```

Like the case in `KafkaTestUtils`:


https://github.com/apache/spark/blob/0d40b1aea758b95a4416c8653599af8713a4aa16/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala#L381

- For the `s.c.mutable.MapOps` type, the `transform` function has also been 
deprecated. At the same time, the signature of `transform` and its replacement 
function `mapValuesInPlace` is as follows:

```scala

  deprecated("Use mapValuesInPlace instead", "2.13.0")
  inline final def transform(f: (K, V) => V): this.type = 
mapValuesInPlace(f)

  def mapValuesInPlace(f: (K, V) => V): this.type = {...}
```

The target type of the value in the function is `V`, which is different 
from the target type of the value in `s.c.immutable.MapOps#transform`, which is 
`W`. This does not meet the desired requirement. So in this scenario, it can be 
divided into two sub-scenarios for handling:

1. If the `mutable.Map` are using needs to be eventually converted to an 
`immutable.Map`, first convert it to an `immutable.Map` and then use the 
`transform` function for replacement. Like the case in `SparkConnectPlanner`:


https://github.com/apache/spark/blob/0d40b1aea758b95a4416c8653599af8713a4aa16/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala#L292

2. If the `mutable.Map` are using does not need to be converted to an 
`immutable.Map` in the end, directly use the `map` function from 
`scala.collection.MapOps` for replacement. Like the case in `SparkSession`:


https://github.com/apache/spark/blob/0d40b1aea758b95a4416c8653599af8713a4aa16/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala#L313

In addition, there is a special case in `PythonWorkerFactory`:


https://github.com/apache/spark/blob/0d40b1aea758b95a4416c8653599af8713a4aa16/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala#L381

For this case, it only needs to `destroy` each `Process` in `values` 
without returning any value. Therefore, it has been rewritten using 
`.values.foreach`.

### Why are the changes needed?
The coding pattern of `s.c.MapOps.view.mapValues` seems verbose, it can be 
simplified using other functions.

### 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 #44122 from LuciferYang/SPARK-46212.

Authored-by: yangjie01 
Signed-off-by: Dongjoon Hyun 
---
 .../spark/util/sketch/CountMinSketchSuite.scala|  2 +-
 .../org/apache/spark/sql/avro/AvroUtils.scala  |  3 +--
 .../scala/org/apache/spark/sql/SparkSession.scala  |  2 +-
 .../spark/sql/ClientDataFrameStatSuite.scala   |  2 +-
 .../org/apache/spark/sql/connect/dsl/package.scala |  2 +-
 .../sql/connect/planner/SparkConnectPlanner.scala  | 15 ++-
 .../sql/kafka010/KafkaMicroBatchSourceSuite.scala  |  2 +-
 .../apache/spark/sql/kafka010/KafkaTestUtils.scala |  3 ++-
 .../streaming/kafka010/ConsumerStrategy.scala  |  9 ---
 .../kafka010/DirectKafkaInputDStream.scala |  2 +-
 .../kafka010/DirectKafkaStreamSuite

<    4   5   6   7   8   9   10   11   12   13   >