(spark) branch master updated: [SPARK-44953][CORE] Log a warning when shuffle tracking is enabled along side another DA supported mechanism

2024-05-13 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 a101c48dd965 [SPARK-44953][CORE] Log a warning when shuffle tracking 
is enabled along side another DA supported mechanism
a101c48dd965 is described below

commit a101c48dd9650d2bca2047b91f9e2a3ba90f142d
Author: zwangsheng 
AuthorDate: Mon May 13 13:33:34 2024 -0700

[SPARK-44953][CORE] Log a warning when shuffle tracking is enabled along 
side another DA supported mechanism

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

Log a warning when shuffle tracking is enabled along side another DA 
supported mechanism

### Why are the changes needed?

Some users enable both shuffle tracking and another mechanism (like 
migration) and then are confused when their jobs don't scale down.

https://issues.apache.org/jira/browse/SPARK-44953

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

Yes, user can find the warning log when enabled both shuffle tracking and 
another DA supported mechanism(shuffle decommission).

### How was this patch tested?

No

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

NO

Closes #45454 from zwangsheng/SPARK-44953.

Authored-by: zwangsheng 
Signed-off-by: Holden Karau 
---
 .../scala/org/apache/spark/ExecutorAllocationManager.scala | 14 +++---
 1 file changed, 11 insertions(+), 3 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index 3bfa1ae0d4dc..1fe02eec3a07 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -206,11 +206,13 @@ private[spark] class ExecutorAllocationManager(
   throw new SparkException(
 s"s${DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT.key} must be > 
0!")
 }
+val shuffleTrackingEnabled = 
conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED)
+val shuffleDecommissionEnabled = decommissionEnabled &&
+  conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)
 if (!conf.get(config.SHUFFLE_SERVICE_ENABLED) && !reliableShuffleStorage) {
-  if (conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING_ENABLED)) {
+  if (shuffleTrackingEnabled) {
 logInfo("Dynamic allocation is enabled without a shuffle service.")
-  } else if (decommissionEnabled &&
-  conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_BLOCKS_ENABLED)) {
+  } else if (shuffleDecommissionEnabled) {
 logInfo("Shuffle data decommission is enabled without a shuffle 
service.")
   } else if (!testing) {
 throw new SparkException("Dynamic allocation of executors requires one 
of the " +
@@ -224,6 +226,12 @@ private[spark] class ExecutorAllocationManager(
   }
 }
 
+if (shuffleTrackingEnabled && (shuffleDecommissionEnabled || 
reliableShuffleStorage)) {
+  logWarning("You are enabling both shuffle tracking and other DA 
supported mechanism, " +
+"which will cause idle executors not to be released in a timely, " +
+"please check the configurations.")
+}
+
 if (executorAllocationRatio > 1.0 || executorAllocationRatio <= 0.0) {
   throw new SparkException(
 s"${DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO.key} must be > 0 and <= 
1.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-45807][SQL] Add createOrReplaceView(..) / replaceView(..) to ViewCatalog

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

holden 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 89673da0beb2 [SPARK-45807][SQL] Add createOrReplaceView(..) / 
replaceView(..) to ViewCatalog
89673da0beb2 is described below

commit 89673da0beb2b64434d29a94e07fa9c6fb4a93e8
Author: Eduard Tudenhoefner 
AuthorDate: Tue Dec 5 10:29:32 2023 -0800

[SPARK-45807][SQL] Add createOrReplaceView(..) / replaceView(..) to 
ViewCatalog

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

ViewCatalog API improvements described in 
[SPIP](https://docs.google.com/document/d/1XOxFtloiMuW24iqJ-zJnDzHl2KMxipTjJoxleJFz66A/edit?usp=sharing)
 that didn't make it into the codebase as part of #37556

### Why are the changes needed?

Required for DataSourceV2 view support.

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

No
### How was this patch tested?

N/A

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

N/A

Closes #43677 from nastra/SPARK-45807.

Authored-by: Eduard Tudenhoefner 
Signed-off-by: Holden Karau 
---
 .../spark/sql/connector/catalog/ViewCatalog.java   | 86 ++
 1 file changed, 86 insertions(+)

diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java
index eb67b9904869..eef348928b2c 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/ViewCatalog.java
@@ -140,6 +140,92 @@ public interface ViewCatalog extends CatalogPlugin {
   String[] columnComments,
   Map properties) throws ViewAlreadyExistsException, 
NoSuchNamespaceException;
 
+  /**
+   * Replace a view in the catalog.
+   * 
+   * The default implementation has a race condition.
+   * Catalogs are encouraged to implement this operation atomically.
+   *
+   * @param ident a view identifier
+   * @param sql the SQL text that defines the view
+   * @param currentCatalog the current catalog
+   * @param currentNamespace the current namespace
+   * @param schema the view query output schema
+   * @param queryColumnNames the query column names
+   * @param columnAliases the column aliases
+   * @param columnComments the column comments
+   * @param properties the view properties
+   * @throws NoSuchViewException If the view doesn't exist or is a table
+   * @throws NoSuchNamespaceException If the identifier namespace does not 
exist (optional)
+   */
+  default void replaceView(
+  Identifier ident,
+  String sql,
+  String currentCatalog,
+  String[] currentNamespace,
+  StructType schema,
+  String[] queryColumnNames,
+  String[] columnAliases,
+  String[] columnComments,
+  Map properties) throws NoSuchViewException, 
NoSuchNamespaceException {
+if (viewExists(ident)) {
+  dropView(ident);
+  try {
+createView(ident, sql, currentCatalog, currentNamespace, schema,
+queryColumnNames, columnAliases, columnComments, properties);
+  }
+  catch (ViewAlreadyExistsException e) {
+throw new RuntimeException("Race condition when dropping and creating 
view", e);
+  }
+} else {
+  throw new NoSuchViewException(ident);
+}
+  }
+
+  /**
+   * Create or replace a view in the catalog.
+   * 
+   * The default implementation has race conditions.
+   * Catalogs are encouraged to implement this operation atomically.
+   *
+   * @param ident a view identifier
+   * @param sql the SQL text that defines the view
+   * @param currentCatalog the current catalog
+   * @param currentNamespace the current namespace
+   * @param schema the view query output schema
+   * @param queryColumnNames the query column names
+   * @param columnAliases the column aliases
+   * @param columnComments the column comments
+   * @param properties the view properties
+   * @throws NoSuchNamespaceException If the identifier namespace does not 
exist (optional)
+   */
+  default void createOrReplaceView(
+  Identifier ident,
+  String sql,
+  String currentCatalog,
+  String[] currentNamespace,
+  StructType schema,
+  String[] queryColumnNames,
+  String[] columnAliases,
+  String[] columnComments,
+  Map properties) throws NoSuchNamespaceException {
+if (viewExists(ident)) {
+  try {
+replaceView(ident, sql, currentCatalog, currentNamespace, schema,
+queryColumnNames, columnAliases, columnComments, properties);
+  } catch (NoSuchViewException e) {
+throw new RuntimeException("Race conditio

[spark] branch master updated: [SPARK-44735][SQL] Add warning msg when inserting columns with the same name by row that don't match up

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

holden 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 ae100267c28b [SPARK-44735][SQL] Add warning msg when inserting columns 
with the same name by row that don't match up
ae100267c28b is described below

commit ae100267c28bc6fd2c2f9c880ed3df1999423992
Author: Jia Fan 
AuthorDate: Mon Oct 23 10:51:31 2023 -0700

[SPARK-44735][SQL] Add warning msg when inserting columns with the same 
name by row that don't match up

### What changes were proposed in this pull request?
This PR add a warning msg when inserting columns name with the same name by 
row but order not matched. Tell user can use `INSERT INTO BY NAME` to reorder 
columns to match with table schema.
It will be like:

![image](https://github.com/apache/spark/assets/32387433/18e57125-8a2e-407c-a3fd-93a9cbf122a1)

### Why are the changes needed?
Optimize user usage scenarios.

### Does this PR introduce _any_ user-facing change?
Yes, sometimes will show some warning.

### How was this patch tested?
Test in local

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

Closes #42763 from Hisoka-X/SPARK-44735_add_warning_for_by_name.

Authored-by: Jia Fan 
Signed-off-by: Holden Karau 
---
 .../apache/spark/sql/catalyst/analysis/Analyzer.scala   |  2 ++
 .../sql/catalyst/analysis/TableOutputResolver.scala | 17 -
 .../apache/spark/sql/execution/datasources/rules.scala  |  5 -
 3 files changed, 22 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 0469fb29a6fc..06d949ece262 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -3421,6 +3421,8 @@ class Analyzer(override val catalogManager: 
CatalogManager) extends RuleExecutor
   case v2Write: V2WriteCommand
   if v2Write.table.resolved && v2Write.query.resolved && 
!v2Write.outputResolved =>
 validateStoreAssignmentPolicy()
+TableOutputResolver.suitableForByNameCheck(v2Write.isByName,
+  expected = v2Write.table.output, queryOutput = v2Write.query.output)
 val projection = TableOutputResolver.resolveOutputColumns(
   v2Write.table.name, v2Write.table.output, v2Write.query, 
v2Write.isByName, conf)
 if (projection != v2Write.query) {
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
index d41757725771..1398552399cd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala
@@ -19,7 +19,9 @@ package org.apache.spark.sql.catalyst.analysis
 
 import scala.collection.mutable
 
+import org.apache.spark.internal.Logging
 import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.SQLConfHelper
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
@@ -34,7 +36,7 @@ import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
 import org.apache.spark.sql.types.{ArrayType, DataType, DecimalType, 
IntegralType, MapType, StructType}
 
-object TableOutputResolver {
+object TableOutputResolver extends SQLConfHelper with Logging {
 
   def resolveVariableOutputColumns(
   expected: Seq[VariableReference],
@@ -470,6 +472,19 @@ object TableOutputResolver {
 }
   }
 
+  def suitableForByNameCheck(
+  byName: Boolean,
+  expected: Seq[Attribute],
+  queryOutput: Seq[Attribute]): Unit = {
+if (!byName && expected.size == queryOutput.size &&
+  expected.forall(e => queryOutput.exists(p => conf.resolver(p.name, 
e.name))) &&
+  expected.zip(queryOutput).exists(e => !conf.resolver(e._1.name, 
e._2.name))) {
+  logWarning("The query columns and the table columns have same names but 
different " +
+"orders. You can use INSERT [INTO | OVERWRITE] BY NAME to reorder the 
query columns to " +
+"align with the table columns.")
+}
+  }
+
   private def containsIntegralOrDecimalType(dt: DataType): Boolean = dt match {
 case _: IntegralType | _: DecimalType => true
 case a: ArrayType => conta

[spark] branch master updated: [SPARK-33782][K8S][CORE] Place spark.files, spark.jars and spark.files under the current working directory on the driver in K8S cluster mode

2022-12-12 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 af8dd411aa9 [SPARK-33782][K8S][CORE] Place spark.files, spark.jars and 
spark.files under the current working directory on the driver in K8S cluster 
mode
af8dd411aa9 is described below

commit af8dd411aa9d15bc59d09cf9959d4a57debc9635
Author: pralabhkumar 
AuthorDate: Mon Dec 12 21:16:54 2022 -0800

[SPARK-33782][K8S][CORE] Place spark.files, spark.jars and spark.files 
under the current working directory on the driver in K8S cluster mode

### What changes were proposed in this pull request?
This PR will place spark.files , spark.jars and spark.pyfiles to the 
current working directory on the driver in K8s cluster mode

### Why are the changes needed?
This mimics the behaviour of Yarn and also helps user to access files from 
PWD . Also as mentioned in the jira
By doing this, users can, for example, leverage PEX to manage Python 
dependences in Apache Spark:
```
pex pyspark==3.0.1 pyarrow==0.15.1 pandas==0.25.3 -o myarchive.pex
PYSPARK_PYTHON=./myarchive.pex spark-submit --files myarchive.pex
```

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

### How was this patch tested?
Tested via unit test cases and also ran on local K8s cluster.

Closes #37417 from pralabhkumar/rk_k8s_local_resource.

Authored-by: pralabhkumar 
Signed-off-by: Holden Karau 
---
 .../org/apache/spark/deploy/SparkSubmit.scala  | 55 +-
 .../org/apache/spark/deploy/SparkSubmitSuite.scala | 35 ++
 2 files changed, 69 insertions(+), 21 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala 
b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 783cf47df16..73acfedd8bc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -20,6 +20,7 @@ package org.apache.spark.deploy
 import java.io._
 import java.lang.reflect.{InvocationTargetException, 
UndeclaredThrowableException}
 import java.net.{URI, URL}
+import java.nio.file.Files
 import java.security.PrivilegedExceptionAction
 import java.text.ParseException
 import java.util.{ServiceLoader, UUID}
@@ -383,43 +384,55 @@ private[spark] class SparkSubmit extends Logging {
   }.orNull
 
   if (isKubernetesClusterModeDriver) {
-// Replace with the downloaded local jar path to avoid propagating 
hadoop compatible uris.
-// Executors will get the jars from the Spark file server.
-// Explicitly download the related files here
-args.jars = localJars
-val filesLocalFiles = Option(args.files).map {
-  downloadFileList(_, targetDir, sparkConf, hadoopConf)
-}.orNull
-val archiveLocalFiles = Option(args.archives).map { uris =>
+// SPARK-33748: this mimics the behaviour of Yarn cluster mode. If the 
driver is running
+// in cluster mode, the archives should be available in the driver's 
current working
+// directory too.
+// SPARK-33782 : This downloads all the files , jars , archiveFiles 
and pyfiles to current
+// working directory
+def downloadResourcesToCurrentDirectory(uris: String, isArchive: 
Boolean = false):
+String = {
   val resolvedUris = Utils.stringToSeq(uris).map(Utils.resolveURI)
-  val localArchives = downloadFileList(
+  val localResources = downloadFileList(
 resolvedUris.map(
   
UriBuilder.fromUri(_).fragment(null).build().toString).mkString(","),
 targetDir, sparkConf, hadoopConf)
-
-  // SPARK-33748: this mimics the behaviour of Yarn cluster mode. If 
the driver is running
-  // in cluster mode, the archives should be available in the driver's 
current working
-  // directory too.
-  
Utils.stringToSeq(localArchives).map(Utils.resolveURI).zip(resolvedUris).map {
-case (localArchive, resolvedUri) =>
-  val source = new File(localArchive.getPath)
+  
Utils.stringToSeq(localResources).map(Utils.resolveURI).zip(resolvedUris).map {
+case (localResources, resolvedUri) =>
+  val source = new File(localResources.getPath)
   val dest = new File(
 ".",
 if (resolvedUri.getFragment != null) resolvedUri.getFragment 
else source.getName)
   logInfo(
-s"Unpacking an archive $resolvedUri " +
+s"Files  $resolvedUri " +
   s"from ${source.getAbsolutePath} to ${dest.getAbsolutePath}")
   Utils.deleteRecursively(dest)
-

[spark] branch master updated: [SPARK-39799][SQL] DataSourceV2: View catalog interface

2022-11-16 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 d12abff7e83 [SPARK-39799][SQL] DataSourceV2: View catalog interface
d12abff7e83 is described below

commit d12abff7e83705fb0e187a79f86b45f99e4b7abb
Author: John Zhuge 
AuthorDate: Wed Nov 16 09:05:14 2022 -0800

[SPARK-39799][SQL] DataSourceV2: View catalog interface

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

ViewCatalog API described in 
[SPIP](https://docs.google.com/document/d/1XOxFtloiMuW24iqJ-zJnDzHl2KMxipTjJoxleJFz66A/edit?usp=sharing).

### Why are the changes needed?

First step towards DataSourceV2 view support.

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

No.

### How was this patch tested?

N/A

Closes #37556 from jzhuge/SPARK-39799.

Authored-by: John Zhuge 
Signed-off-by: Holden Karau 
---
 core/src/main/resources/error/error-classes.json   |  15 ++
 .../apache/spark/sql/connector/catalog/View.java   |  74 
 .../spark/sql/connector/catalog/ViewCatalog.java   | 188 +
 .../spark/sql/connector/catalog/ViewChange.java|  79 +
 .../catalyst/analysis/AlreadyExistException.scala  |  10 ++
 .../spark/sql/catalyst/analysis/Analyzer.scala |   2 +-
 .../catalyst/analysis/NoSuchItemException.scala|   8 +
 7 files changed, 375 insertions(+), 1 deletion(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index d5d6e938ad1..c96cb11874c 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1428,6 +1428,21 @@
   "3. set \"spark.sql.legacy.allowUntypedScalaUDF\" to \"true\" and use 
this API with caution"
 ]
   },
+  "VIEW_ALREADY_EXISTS" : {
+"message" : [
+  "Cannot create view  because it already exists.",
+  "Choose a different name, drop or replace the existing object, or add 
the IF NOT EXISTS clause to tolerate pre-existing objects."
+],
+"sqlState" : "42000"
+  },
+  "VIEW_NOT_FOUND" : {
+"message" : [
+  "The view  cannot be found. Verify the spelling and 
correctness of the schema and catalog.",
+  "If you did not qualify the name with a schema, verify the 
current_schema() output, or qualify the name with the correct schema and 
catalog.",
+  "To tolerate the error on drop use DROP VIEW IF EXISTS."
+],
+"sqlState" : "42000"
+  },
   "_LEGACY_ERROR_TEMP_0001" : {
 "message" : [
   "Invalid InsertIntoContext"
diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/View.java 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/View.java
new file mode 100644
index 000..a4dc5f2f2d2
--- /dev/null
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/View.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.connector.catalog;
+
+import java.util.Map;
+
+import org.apache.spark.annotation.DeveloperApi;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * An interface representing a persisted view.
+ */
+@DeveloperApi
+public interface View {
+  /**
+   * A name to identify this view.
+   */
+  String name();
+
+  /**
+   * The view query SQL text.
+   */
+  String query();
+
+  /**
+   * The current catalog when the view is created.
+   */
+  String currentCatalog();
+
+  /**
+   * The current namespace when the view is created.
+   */
+  String[] currentNamespace();
+
+  /**
+   * The schema for the view when the view is created after applying column 
aliases.
+   */
+  StructType schema();
+
+  /**
+   * The output column names of the query that creates this view.
+   */
+  String[] queryColumnNames();
+
+  /**
+   * The view colu

[spark] branch master updated: [SPARK-40428][CORE] Fix shutdown hook in the CoarseGrainedSchedulerBackend

2022-10-04 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 505a8a04ad6 [SPARK-40428][CORE] Fix shutdown hook in the 
CoarseGrainedSchedulerBackend
505a8a04ad6 is described below

commit 505a8a04ad64a6732bf9fec03c28bfbd514d109d
Author: Holden Karau 
AuthorDate: Tue Oct 4 17:26:34 2022 -0700

[SPARK-40428][CORE] Fix shutdown hook in the CoarseGrainedSchedulerBackend

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

Fix the shutdown hook call through to CoarseGrainedSchedulerBackend

### Why are the changes needed?

Sometimes if the driver shuts down abnormally resources may be left 
dangling.

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

No

### How was this patch tested?

Existing tests.

Closes #37885 from holdenk/shutdownhook-for-k8s.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 core/src/main/scala/org/apache/spark/SparkContext.scala  |  4 +++-
 .../scala/org/apache/spark/scheduler/DAGScheduler.scala  | 16 
 .../org/apache/spark/scheduler/TaskSchedulerImpl.scala   | 16 
 3 files changed, 27 insertions(+), 9 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala 
b/core/src/main/scala/org/apache/spark/SparkContext.scala
index f101dc8e083..e2c6a912bc2 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -2119,7 +2119,9 @@ class SparkContext(config: SparkConf) extends Logging {
 Utils.tryLogNonFatalError {
   _plugins.foreach(_.shutdown())
 }
-FallbackStorage.cleanUp(_conf, _hadoopConfiguration)
+Utils.tryLogNonFatalError {
+  FallbackStorage.cleanUp(_conf, _hadoopConfiguration)
+}
 Utils.tryLogNonFatalError {
   _eventLogger.foreach(_.stop())
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index ab05409aebb..c5529851382 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -2890,10 +2890,18 @@ private[spark] class DAGScheduler(
   }
 
   def stop(): Unit = {
-messageScheduler.shutdownNow()
-shuffleMergeFinalizeScheduler.shutdownNow()
-eventProcessLoop.stop()
-taskScheduler.stop()
+Utils.tryLogNonFatalError {
+  messageScheduler.shutdownNow()
+}
+Utils.tryLogNonFatalError {
+  shuffleMergeFinalizeScheduler.shutdownNow()
+}
+Utils.tryLogNonFatalError {
+  eventProcessLoop.stop()
+}
+Utils.tryLogNonFatalError {
+  taskScheduler.stop()
+}
   }
 
   eventProcessLoop.start()
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index a6735f380f1..5004262a71c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -972,15 +972,23 @@ private[spark] class TaskSchedulerImpl(
   }
 
   override def stop(): Unit = {
-speculationScheduler.shutdown()
+Utils.tryLogNonFatalError {
+  speculationScheduler.shutdown()
+}
 if (backend != null) {
-  backend.stop()
+  Utils.tryLogNonFatalError {
+backend.stop()
+  }
 }
 if (taskResultGetter != null) {
-  taskResultGetter.stop()
+  Utils.tryLogNonFatalError {
+taskResultGetter.stop()
+  }
 }
 if (barrierCoordinator != null) {
-  barrierCoordinator.stop()
+  Utils.tryLogNonFatalError {
+barrierCoordinator.stop()
+  }
 }
 starvationTimer.cancel()
 abortTimer.cancel()


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



[spark] branch master updated: [SPARK-38969][K8S] Fix Decom reporting

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

holden 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 451ac4c4d96 [SPARK-38969][K8S] Fix Decom reporting
451ac4c4d96 is described below

commit 451ac4c4d96f08df8176d9b7c872ab8968623b69
Author: Holden Karau 
AuthorDate: Fri Aug 12 16:29:47 2022 -0700

[SPARK-38969][K8S] Fix Decom reporting

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

Change how we account for executor loss reasons.

### Why are the changes needed?

Race condition in executors which decommission quickly.

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

No

### How was this patch tested?

Existing core tests.

Closes #36434 from holdenk/SPARK-38969-decom-reporting.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../apache/spark/scheduler/dynalloc/ExecutorMonitor.scala | 10 --
 .../kubernetes/docker/src/main/dockerfiles/spark/decom.sh | 15 ++-
 2 files changed, 14 insertions(+), 11 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala 
b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala
index defef5bfcf2..9132d0e46d3 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala
@@ -355,13 +355,11 @@ private[spark] class ExecutorMonitor(
 val removed = executors.remove(event.executorId)
 if (removed != null) {
   decrementExecResourceProfileCount(removed.resourceProfileId)
-  if (removed.decommissioning) {
-if (event.reason == ExecutorLossMessage.decommissionFinished ||
-event.reason == ExecutorDecommission().message) {
-  metrics.gracefullyDecommissioned.inc()
-} else {
+  if (event.reason == ExecutorLossMessage.decommissionFinished ||
+event.reason == ExecutorDecommission().message) {
+metrics.gracefullyDecommissioned.inc()
+  } else if (removed.decommissioning) {
   metrics.decommissionUnfinished.inc()
-}
   } else if (removed.pendingRemoval) {
 metrics.driverKilled.inc()
   } else {
diff --git 
a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/decom.sh 
b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/decom.sh
index cd973df257f..4c25b42e964 100755
--- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/decom.sh
+++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/decom.sh
@@ -18,17 +18,22 @@
 #
 
 
-set -ex
+set +e
+set -x
 echo "Asked to decommission"
 # Find the pid to signal
 date | tee -a ${LOG}
-WORKER_PID=$(ps -o pid -C java | tail -n 1| awk '{ sub(/^[ \t]+/, ""); print 
}')
+WORKER_PID=$(ps -o pid,cmd -C java |grep Executor \
+  | tail -n 1| awk '{ sub(/^[ \t]+/, ""); print }' \
+  | cut -f 1 -d " ")
 echo "Using worker pid $WORKER_PID"
 kill -s SIGPWR ${WORKER_PID}
-# For now we expect this to timeout, since we don't start exiting the backend.
+# If the worker does exit stop blocking K8s cleanup. Note this is a "soft"
+# block since the pod it's self will have a maximum decommissioning time which 
will
+# overload this.
 echo "Waiting for worker pid to exit"
-# If the worker does exit stop blocking the cleanup.
-timeout 60 tail --pid=${WORKER_PID} -f /dev/null
+tail --pid=${WORKER_PID} -f /dev/null
+sleep 1
 date
 echo "Done"
 date


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



[spark] branch master updated: [SPARK-36664][CORE] Log time waiting for cluster resources

2022-04-25 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 148918b5469 [SPARK-36664][CORE] Log time waiting for cluster resources
148918b5469 is described below

commit 148918b5469eb972b5d8e22d9b1cea5ca0721045
Author: Holden Karau 
AuthorDate: Mon Apr 25 20:42:37 2022 -0700

[SPARK-36664][CORE] Log time waiting for cluster resources

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

Keep track of and communicate to the listener bus how long we are waiting 
for execs to be allocated from the underlying cluster manager.

Replaces previous PR with GHA issues (  #35172 ) and WIP PR 
https://github.com/apache/spark/pull/34650  and #35881

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

Keep track of and communicate to the listener bus how long we are waiting 
for execs to be allocated from the underlying cluster manager.

Replaces previous PR with GHA issues (  #35172 ) and WIP PR 
https://github.com/apache/spark/pull/34650

### Why are the changes needed?

Sometimes the cluster manager may choke or otherwise not be able to 
allocate resources and we don't have a good way of detecting this situation 
making it difficult for the user to debug and tell apart from Spark not scaling 
up correctly.

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

New field in the listener bus message for when a executor is allocated.

### How was this patch tested?

New unit test in the listener suite.
### Why are the changes needed?

Sometimes the cluster manager may choke or otherwise not be able to 
allocate resources and we don't have a good way of detecting this situation 
making it difficult for the user to debug and tell apart from Spark not scaling 
up correctly.

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

New field in the listener bus message for when a executor is allocated.

### How was this patch tested?

New unit test in the listener suite.

Closes #36185 from 
holdenk/SPARK-36664-Log-time-waiting-for-cluster-resources-r4.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../cluster/CoarseGrainedSchedulerBackend.scala|  67 ++-
 .../spark/scheduler/cluster/ExecutorData.scala |   6 +-
 .../spark/scheduler/cluster/ExecutorInfo.scala |  20 +++-
 .../scala/org/apache/spark/util/JsonProtocol.scala |  13 +-
 .../CoarseGrainedSchedulerBackendSuite.scala   | 131 +
 .../scheduler/SparkListenerWithClusterSuite.scala  |   1 +
 .../scheduler/dynalloc/ExecutorMonitorSuite.scala  |   2 +-
 .../org/apache/spark/util/JsonProtocolSuite.scala  |  38 ++
 8 files changed, 266 insertions(+), 12 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 13a7183a29d..61d67765c8c 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -21,7 +21,7 @@ import java.util.concurrent.{ScheduledExecutorService, 
TimeUnit}
 import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
 import javax.annotation.concurrent.GuardedBy
 
-import scala.collection.mutable.{HashMap, HashSet}
+import scala.collection.mutable.{HashMap, HashSet, Queue}
 import scala.concurrent.Future
 
 import org.apache.hadoop.security.UserGroupInformation
@@ -82,6 +82,12 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   @GuardedBy("CoarseGrainedSchedulerBackend.this")
   private val requestedTotalExecutorsPerResourceProfile = new 
HashMap[ResourceProfile, Int]
 
+  // Profile IDs to the times that executors were requested for.
+  // The operations we do on queue are all amortized constant cost
+  // see 
https://www.scala-lang.org/api/2.13.x/scala/collection/mutable/ArrayDeque.html
+  @GuardedBy("CoarseGrainedSchedulerBackend.this")
+  private val execRequestTimes = new HashMap[Int, Queue[(Int, Long)]]
+
   private val listenerBus = scheduler.sc.listenerBus
 
   // Executors we have requested the cluster manager to kill that have not 
died yet; maps
@@ -260,9 +266,27 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   
.resourceProfileFromId(resourceProfileId).getNumSlotsPerAddress(rName, conf)
 (info.name, new ExecutorResourceInfo(info.name, info.addresses, 
numParts))
   }
+  // If we've requested the executor figure out when we did.
+  va

[spark] branch master updated: [SPARK-38561][K8S][DOCS] Add doc for `Customized Kubernetes Schedulers`

2022-03-16 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 4ff40c1  [SPARK-38561][K8S][DOCS] Add doc for `Customized Kubernetes 
Schedulers`
4ff40c1 is described below

commit 4ff40c10f02f6e0735ce6554f7338489d8555bce
Author: Yikun Jiang 
AuthorDate: Wed Mar 16 11:12:54 2022 -0700

[SPARK-38561][K8S][DOCS] Add doc for `Customized Kubernetes Schedulers`

### What changes were proposed in this pull request?
This is PR to doc for basic framework capability for Customized Kubernetes 
Schedulers.

### Why are the changes needed?
Guide user how to use spark on kubernetes custom scheduler

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

### How was this patch tested?
CI passed

Closes #35869 from Yikun/SPARK-38561.

Authored-by: Yikun Jiang 
Signed-off-by: Holden Karau 
---
 docs/running-on-kubernetes.md | 19 +++
 1 file changed, 19 insertions(+)

diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md
index de37e22..d1b2fcd 100644
--- a/docs/running-on-kubernetes.md
+++ b/docs/running-on-kubernetes.md
@@ -1713,6 +1713,25 @@ spec:
 image: will-be-overwritten
 ```
 
+ Customized Kubernetes Schedulers for Spark on Kubernetes
+
+Spark allows users to specify a custom Kubernetes schedulers.
+
+1. Specify scheduler name.
+
+   Users can specify a custom scheduler using 
spark.kubernetes.scheduler.name or
+   spark.kubernetes.{driver/executor}.scheduler.name 
configuration.
+
+2. Specify scheduler related configurations.
+
+   To configure the custom scheduler the user can use [Pod 
templates](#pod-template), add labels 
(spark.kubernetes.{driver,executor}.label.*)  and/or annotations 
(spark.kubernetes.{driver/executor}.annotation.*).
+
+3. Specify scheduler feature step.
+
+   Users may also consider to use 
spark.kubernetes.{driver/executor}.pod.featureSteps to support 
more complex requirements, including but not limited to:
+  - Create additional Kubernetes custom resources for driver/executor 
scheduling.
+  - Set scheduler hints according to configuration or existing Pod info 
dynamically.
+
 ### Stage Level Scheduling Overview
 
 Stage level scheduling is supported on Kubernetes when dynamic allocation is 
enabled. This also requires 
spark.dynamicAllocation.shuffleTracking.enabled to be enabled 
since Kubernetes doesn't support an external shuffle service at this time. The 
order in which containers for different profiles is requested from Kubernetes 
is not guaranteed. Note that since dynamic allocation on Kubernetes requires 
the shuffle tracking feature, this means that executors from previous stages t 
[...]

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



[spark-website] branch asf-site updated: Fix dockerhub link

2022-02-21 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new d0d3def  Fix dockerhub link
d0d3def is described below

commit d0d3defa2e20ec29ffae4a48702131a2953885ea
Author: Holden Karau 
AuthorDate: Mon Feb 21 14:01:42 2022 -0800

Fix dockerhub link
---
 downloads.md| 2 +-
 site/downloads.html | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/downloads.md b/downloads.md
index 8a6aca7..bf5a98e 100644
--- a/downloads.md
+++ b/downloads.md
@@ -43,7 +43,7 @@ Spark artifacts are [hosted in Maven 
Central](https://search.maven.org/search?q=
 
 ### Convenience Docker Container Images
 
-[Spark Docker Container images are available from 
DockerHub](https://hub.docker.com/repository/docker/apache/spark), these images 
contain non-ASF software and may be subject to different license terms.
+[Spark Docker Container images are available from 
DockerHub](https://hub.docker.com/r/apache/spark-py/tags), these images contain 
non-ASF software and may be subject to different license terms.
 
 ### Release notes for stable releases
 
diff --git a/site/downloads.html b/site/downloads.html
index e8ecb9c..441eb85 100644
--- a/site/downloads.html
+++ b/site/downloads.html
@@ -187,7 +187,7 @@ version: 3.2.1
 
 Convenience Docker Container 
Images
 
-https://hub.docker.com/repository/docker/apache/spark;>Spark 
Docker Container images are available from DockerHub, these images contain 
non-ASF software and may be subject to different license terms.
+https://hub.docker.com/r/apache/spark-py/tags;>Spark Docker 
Container images are available from DockerHub, these images contain non-ASF 
software and may be subject to different license terms.
 
 Release notes for stable 
releases
 

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



[spark-website] branch asf-site updated (08c07ea -> e631332)

2022-02-21 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a change to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git.


from 08c07ea  Regenerate site.
 new a5c41b0  Fix 3.1.3 doc link
 new e631332  Fix detailed change link for Spark 3.1.3

The 2 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:
 documentation.md  | 2 +-
 releases/_posts/2022-02-18-spark-release-3-1-3.md | 2 +-
 site/documentation.html   | 2 +-
 site/releases/spark-release-3-1-3.html| 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)

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



[spark-website] 01/02: Fix 3.1.3 doc link

2022-02-21 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git

commit a5c41b0d1f491585dea3d89c3aca660efa954446
Author: Holden Karau 
AuthorDate: Mon Feb 21 11:58:03 2022 -0800

Fix 3.1.3 doc link
---
 documentation.md| 2 +-
 site/documentation.html | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/documentation.md b/documentation.md
index 3a443cd..c5ebb64 100644
--- a/documentation.md
+++ b/documentation.md
@@ -14,7 +14,7 @@ navigation:
 
   Spark 3.2.1
   Spark 3.2.0
-  Spark 3.1.3
+  Spark 3.1.3
   Spark 3.1.2
   Spark 3.1.1
   Spark 3.0.3
diff --git a/site/documentation.html b/site/documentation.html
index c1f98a6..07eefb1 100644
--- a/site/documentation.html
+++ b/site/documentation.html
@@ -151,7 +151,7 @@
 
   Spark 3.2.1
   Spark 3.2.0
-  Spark 3.1.3
+  Spark 3.1.3
   Spark 3.1.2
   Spark 3.1.1
   Spark 3.0.3

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



[spark-website] 02/02: Fix detailed change link for Spark 3.1.3

2022-02-21 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git

commit e631332c8ac7ee45b0e1cd4d3ad34320a7bc9124
Author: Holden Karau 
AuthorDate: Mon Feb 21 12:01:20 2022 -0800

Fix detailed change link for Spark 3.1.3
---
 releases/_posts/2022-02-18-spark-release-3-1-3.md | 2 +-
 site/releases/spark-release-3-1-3.html| 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/releases/_posts/2022-02-18-spark-release-3-1-3.md 
b/releases/_posts/2022-02-18-spark-release-3-1-3.md
index b48b19b..25a0417 100644
--- a/releases/_posts/2022-02-18-spark-release-3-1-3.md
+++ b/releases/_posts/2022-02-18-spark-release-3-1-3.md
@@ -36,6 +36,6 @@ While being a maintence release we did still upgrade some 
dependencies in this r
   - [[SPARK-36129]](https://issues.apache.org/jira/browse/SPARK-36129): 
Upgrade commons-compress to 1.21
 
 
-You can consult JIRA for the [detailed 
changes](https://s.apache.org/spark-3.2.1).
+You can consult JIRA for the [detailed 
changes](https://s.apache.org/spark-3.1.3).
 
 We would like to acknowledge all community members for contributing patches to 
this release.
diff --git a/site/releases/spark-release-3-1-3.html 
b/site/releases/spark-release-3-1-3.html
index 1ca1fac..1176ff6 100644
--- a/site/releases/spark-release-3-1-3.html
+++ b/site/releases/spark-release-3-1-3.html
@@ -175,7 +175,7 @@
   https://issues.apache.org/jira/browse/SPARK-36129;>[SPARK-36129]: 
Upgrade commons-compress to 1.21
 
 
-You can consult JIRA for the https://s.apache.org/spark-3.2.1;>detailed changes.
+You can consult JIRA for the https://s.apache.org/spark-3.1.3;>detailed changes.
 
 We would like to acknowledge all community members for contributing patches 
to this release.
 

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



[spark-website] 01/03: Initial 3.1.3 addition

2022-02-21 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git

commit cce59bf813570795e1d25d42a98e7c06de413ec5
Author: Holden Karau 
AuthorDate: Fri Feb 18 17:24:39 2022 -0800

Initial 3.1.3 addition
---
 documentation.md  |  1 +
 downloads.md  |  5 +++
 js/downloads.js   |  2 +-
 news/_posts/2022-02-18-3-1-3-released.md  | 14 
 releases/_posts/2022-02-18-spark-release-3-1-3.md | 41 +++
 5 files changed, 62 insertions(+), 1 deletion(-)

diff --git a/documentation.md b/documentation.md
index eeab8c4..3a443cd 100644
--- a/documentation.md
+++ b/documentation.md
@@ -14,6 +14,7 @@ navigation:
 
   Spark 3.2.1
   Spark 3.2.0
+  Spark 3.1.3
   Spark 3.1.2
   Spark 3.1.1
   Spark 3.0.3
diff --git a/downloads.md b/downloads.md
index e5d1db3..8a6aca7 100644
--- a/downloads.md
+++ b/downloads.md
@@ -40,6 +40,11 @@ Spark artifacts are [hosted in Maven 
Central](https://search.maven.org/search?q=
 ### Installing with PyPi
 https://pypi.org/project/pyspark/;>PySpark is now available in 
pypi. To install just run `pip install pyspark`.
 
+
+### Convenience Docker Container Images
+
+[Spark Docker Container images are available from 
DockerHub](https://hub.docker.com/repository/docker/apache/spark), these images 
contain non-ASF software and may be subject to different license terms.
+
 ### Release notes for stable releases
 
 
diff --git a/js/downloads.js b/js/downloads.js
index 2d40312..babc72a 100644
--- a/js/downloads.js
+++ b/js/downloads.js
@@ -27,7 +27,7 @@ var packagesV11 = [hadoop3p2, hadoop2p7, hadoopFree, sources];
 var packagesV12 = [hadoop3p3, hadoop3p3scala213, hadoop2p7, hadoopFree, 
sources];
 
 addRelease("3.2.1", new Date("01/26/2022"), packagesV12, true);
-addRelease("3.1.2", new Date("06/01/2021"), packagesV11, true);
+addRelease("3.1.3", new Date("02/18/2022"), packagesV11, true);
 addRelease("3.0.3", new Date("06/23/2021"), packagesV10, true);
 
 function append(el, contents) {
diff --git a/news/_posts/2022-02-18-3-1-3-released.md 
b/news/_posts/2022-02-18-3-1-3-released.md
new file mode 100644
index 000..d9a826f
--- /dev/null
+++ b/news/_posts/2022-02-18-3-1-3-released.md
@@ -0,0 +1,14 @@
+---
+layout: post
+title: Spark 3.1.3 released
+categories:
+- News
+tags: []
+status: publish
+type: post
+published: true
+meta:
+  _edit_last: '4'
+  _wpas_done_all: '1'
+---
+We are happy to announce the availability of Spark 3.1.3! Visit the release notes to read about the new features, or download the release today.
diff --git a/releases/_posts/2022-02-18-spark-release-3-1-3.md 
b/releases/_posts/2022-02-18-spark-release-3-1-3.md
new file mode 100644
index 000..b48b19b
--- /dev/null
+++ b/releases/_posts/2022-02-18-spark-release-3-1-3.md
@@ -0,0 +1,41 @@
+---
+layout: post
+title: Spark Release 3.1.3
+categories: []
+tags: []
+status: publish
+type: post
+published: true
+meta:
+_edit_last: '4'
+_wpas_done_all: '1'
+---
+
+Spark 3.1.3 is a maintenance release containing stability fixes. This release 
is based on the branch-3.1 maintenance branch of Spark. We strongly recommend 
all 3.1.3 users to upgrade to this stable release.
+
+### Notable changes
+
+  - [[SPARK-38075]](https://issues.apache.org/jira/browse/SPARK-38075): Hive 
script transform with order by and limit will return fake rows
+  - [[SPARK-37784]](https://issues.apache.org/jira/browse/SPARK-37784): 
CodeGenerator.addBufferedState() does not properly handle UDTs
+  - [[SPARK-37079]](https://issues.apache.org/jira/browse/SPARK-37079): Fix 
DataFrameWriterV2.partitionedBy to send the arguments to JVM properly
+  - [[SPARK-37049]](https://issues.apache.org/jira/browse/SPARK-37049): 
executorIdleTimeout is not working for pending pods on K8s
+  - [[SPARK-36783]](https://issues.apache.org/jira/browse/SPARK-36783): 
ScanOperation should not push Filter through nondeterministic Project
+  - [[SPARK-36782]](https://issues.apache.org/jira/browse/SPARK-36782): 
Deadlock between map-output-dispatcher and dispatcher-BlockManagerMaster upon 
migrating shuffle blocks
+  - [[SPARK-36704]](https://issues.apache.org/jira/browse/SPARK-36704): 
InaccessibleObjectException in Java 9+ on startup
+  - [[SPARK-36740]](https://issues.apache.org/jira/browse/SPARK-36740): 
Various NaN issues with array operations.
+  - [[SPARK-36532]](https://issues.apache.org/jira/browse/SPARK-36532): 
Deadlock in CoarseGrainedExecutorBackend.onDisconnected
+  - [[SPARK-36489]](https://issues.apache.org/jira/browse/SPARK-36489): 
Aggregate functions over no grouping keys, on tables with a single bucket, 
return multiple rows
+  - [[SPARK-36339]](https://issues.apache.org/jira/browse/SPARK-36339): 
aggsBuffer should collect AggregateExpressio

svn commit: r52683 - in /dev/spark: v3.1.3-rc1-bin/ v3.1.3-rc2-bin/ v3.1.3-rc2-docs/ v3.1.3-rc3-bin/ v3.1.3-rc3-docs/ v3.1.3-rc4-docs/

2022-02-21 Thread holden
Author: holden
Date: Mon Feb 21 19:39:18 2022
New Revision: 52683

Log:
Removing RC artifacts.

Removed:
dev/spark/v3.1.3-rc1-bin/
dev/spark/v3.1.3-rc2-bin/
dev/spark/v3.1.3-rc2-docs/
dev/spark/v3.1.3-rc3-bin/
dev/spark/v3.1.3-rc3-docs/
dev/spark/v3.1.3-rc4-docs/


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



[spark] tag v3.1.3 created (now d1f8a50)

2022-02-21 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


  at d1f8a50  (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: r52631 - /dev/spark/v3.1.3-rc4-bin/ /release/spark/spark-3.1.3/

2022-02-18 Thread holden
Author: holden
Date: Fri Feb 18 22:56:11 2022
New Revision: 52631

Log:
Release Spark 3.1.3


Added:
release/spark/spark-3.1.3/
  - copied from r52630, dev/spark/v3.1.3-rc4-bin/
Removed:
dev/spark/v3.1.3-rc4-bin/


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



svn commit: r52419 - in /dev/spark/v3.1.3-rc4-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _site/api/java/org/apache/parqu

2022-02-06 Thread holden
Author: holden
Date: Mon Feb  7 00:26:19 2022
New Revision: 52419

Log:
Apache Spark v3.1.3-rc4 docs


[This commit notification would consist of 2264 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: r52418 - /dev/spark/v3.1.3-rc4-bin/

2022-02-06 Thread holden
Author: holden
Date: Sun Feb  6 23:22:00 2022
New Revision: 52418

Log:
Apache Spark v3.1.3-rc4

Added:
dev/spark/v3.1.3-rc4-bin/
dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz   (with props)
dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz.asc
dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz.sha512
dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz   (with props)
dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz.asc
dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz.sha512
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-hadoop2.7.tgz   (with props)
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-hadoop2.7.tgz.asc
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-hadoop2.7.tgz.sha512
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-hadoop3.2.tgz   (with props)
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-hadoop3.2.tgz.asc
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-hadoop3.2.tgz.sha512
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-without-hadoop.tgz   (with props)
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-without-hadoop.tgz.asc
dev/spark/v3.1.3-rc4-bin/spark-3.1.3-bin-without-hadoop.tgz.sha512
dev/spark/v3.1.3-rc4-bin/spark-3.1.3.tgz   (with props)
dev/spark/v3.1.3-rc4-bin/spark-3.1.3.tgz.asc
dev/spark/v3.1.3-rc4-bin/spark-3.1.3.tgz.sha512

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

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

Added: dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz.asc
==
--- dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz.asc (added)
+++ dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz.asc Sun Feb  6 23:22:00 2022
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJGBAABCgAwFiEEyhnizCAbZSLToqBdxBq2dEJPw/sFAmIAVCcSHGhvbGRlbkBh
+cGFjaGUub3JnAAoJEMQatnRCT8P7UjoP/3BqiMyJ6hihcN1vhi1no2/IXeE2xCPF
+uOeqZCiTtKzHK6mPSFwwCysZJCdsnbLVZDs/ZKncN9azJS5tc626HUKxD6CTXOk6
+F6U/BcH9WXacWZDPNqriZ00ZqT5UYU4MEUeqnu5TDMvRN4lF9Bcl0qqKAUFC/naA
+eZ/cYxnLWUKmmPU93oVFX0JxMlIW9KJCX+UI3S5zzNGq5vbX8cwTx7CE4ks+10xh
+V3RvI6KDyJ3Iimt62dpkAROtioqgI3d9x72KgcUYpQaI6J8ZvOIuAEDfRSY+yAYA
+/qNXhZuvXQHxpu2A6ubMD4h9aLijApqH3LMPWW4lfIi81NKLlfkK6P2847cTXRvG
+BNq9EYC8ANu64eFeItxXb3vWw2fcR0PcGzyRfTwkJgycunzj2mmS744xmKltZNjf
+FGWVxInmBOC3j3C1zgVIX2moZu8ywyUF3HWWb9Jgc/hSYl/wtTJBTd99NHDz+Axh
+czPSvt3195JRh0JhtW8WECcNl08aa7hoQ68r24tWfjEVvwMTRqr4t161mQs2iyje
++aHx8bg9u9oKYy18ScADUOb2YKRkFpx9U5UWvgyKmWNVU8w0d8FBgu73J1dVLQXQ
+aXWO1f2Uflu+gsmO6yNxzRGHXMP+a1/kUmqrFhWP7z9zX8VIxXnb035YPxjqWGpP
+VRqqVJEMXl1Y
+=/Qwl
+-END PGP SIGNATURE-

Added: dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz.sha512
==
--- dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz.sha512 (added)
+++ dev/spark/v3.1.3-rc4-bin/SparkR_3.1.3.tar.gz.sha512 Sun Feb  6 23:22:00 2022
@@ -0,0 +1,3 @@
+SparkR_3.1.3.tar.gz: 6F4FC494 8CCFCC27 20AD34BE 5F30FF5A 1035B48E 7934E955
+ C2A00A66 37262DB3 BEE34E9C 34B6DE09 7B41E85E 197AE593
+ 3692F598 BDA6D3F6 137F6DBE 780454DE

Added: dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz.asc
==
--- dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz.asc (added)
+++ dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz.asc Sun Feb  6 23:22:00 2022
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJGBAABCgAwFiEEyhnizCAbZSLToqBdxBq2dEJPw/sFAmIAVCgSHGhvbGRlbkBh
+cGFjaGUub3JnAAoJEMQatnRCT8P7uBQP/iKk+W1vtQMlAeYXY/WyoiHjyPNc+kCj
+C7MAgDt/rlj0VBoxSJ9R6l6aw+5EYIv/vZDo52tvpAdxk69w9zxj4I+8U/HnwbHw
+8DoPbMqfhl4cWLFLH5HXadGjuDRfs7ULeXv6Sz0KMnj+UtISVMBu9KhUeHOt/tyw
+ZwM8d53XzgsrjcqR6drEBgbzUM3C1M3qLAt2hoftcYQ4OlNKBixJ3/o6t5nVGPbQ
+PCRbNL/iBNGntPNEO3XRjyuDDkqvj3h2VM5miCk8E1mUKIoJfj4zBx4K48qjKDPk
+4e6fMZocehdFQ1l1Z03hH/9ZPSRcJIrItMWlacmtfi35FASoKLPLwh88UwMazFCU
+NqSHq/n8iZX9tj9F8uSSg4ME8uEr0Uo+nmifkTWLMWGfWXU4FdvhjkjqnWyS+JFx
+iJHPtmJq98XJFehdcPTh+3dWpPhLL2nRD/bJwlHF2kGMOGUPGqHp5S3XgjuebYw0
+Ofr9kKs9LoA8Rd9XAFNsc2U1KyUdFz2Sq+lYwivXv1f1MtvAMyc2/rBsD89S7xZF
+plcDWGqAiYQZAI47VQmQk9KAOBIIT6gtirHNtFvRDTTqOtn49tsKOXk4tLiFOA+8
+bJ93YLmDStTMZWBdPqUUqQreN96dCazfk+sjWEeT9HUx2dl+5ZBuTnlIOcaoJNuh
+T5RG6gWmVldT
+=HtZc
+-END PGP SIGNATURE-

Added: dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3.tar.gz.sha512
==
--- dev/spark/v3.1.3-rc4-bin/pyspark-3.1.3

[spark] 01/01: Preparing development version 3.1.4-SNAPSHOT

2022-02-06 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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

commit 0b35c2464f805970d2a62ccc757616332287eb2b
Author: Holden Karau 
AuthorDate: Sun Feb 6 22:17:37 2022 +

Preparing development version 3.1.4-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   | 4 ++--
 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, 40 insertions(+), 40 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index d42841c..db689ea 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.1.3
+Version: 3.1.4
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = "aut",
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 95309ee..8e2cded 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 80f6630..baf20e6 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index ead7a4f..1a9c791 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 232abd7..8a188a3 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 79068a6..d53f21d 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 2030f85..53b7312 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/tags

[spark] branch branch-3.1 updated (7540421 -> 0b35c24)

2022-02-06 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


from 7540421  Preparing development version 3.1.4-SNAPSHOT
 add d1f8a50  Preparing Spark release v3.1.3-rc4
 new 0b35c24  Preparing development version 3.1.4-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:

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



[spark] 01/01: Preparing Spark release v3.1.3-rc4

2022-02-06 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to tag v3.1.3-rc4
in repository https://gitbox.apache.org/repos/asf/spark.git

commit d1f8a503a26bcfb4e466d9accc5fa241a7933667
Author: Holden Karau 
AuthorDate: Sun Feb 6 22:17:33 2022 +

Preparing Spark release v3.1.3-rc4
---
 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   | 4 ++--
 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, 40 insertions(+), 40 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index db689ea..d42841c 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.1.4
+Version: 3.1.3
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = "aut",
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 8e2cded..95309ee 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index baf20e6..80f6630 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 1a9c791..ead7a4f 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 8a188a3..232abd7 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index d53f21d..79068a6 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 53b7312..2030f85 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/common

[spark] tag v3.1.3-rc4 created (now d1f8a50)

2022-02-06 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a change to tag v3.1.3-rc4
in repository https://gitbox.apache.org/repos/asf/spark.git.


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

 new d1f8a50  Preparing Spark release v3.1.3-rc4

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



svn commit: r52357 - in /dev/spark/v3.1.3-rc3-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _site/api/java/org/apache/parqu

2022-02-01 Thread holden
Author: holden
Date: Tue Feb  1 22:41:57 2022
New Revision: 52357

Log:
Apache Spark v3.1.3-rc3 docs


[This commit notification would consist of 2264 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: r52356 - /dev/spark/v3.1.3-rc3-bin/

2022-02-01 Thread holden
Author: holden
Date: Tue Feb  1 22:17:13 2022
New Revision: 52356

Log:
Apache Spark v3.1.3-rc3

Added:
dev/spark/v3.1.3-rc3-bin/
dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz   (with props)
dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz.asc
dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz.sha512
dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz   (with props)
dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz.asc
dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz.sha512
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-hadoop2.tgz   (with props)
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-hadoop2.tgz.asc
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-hadoop2.tgz.sha512
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-hadoop3.tgz   (with props)
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-hadoop3.tgz.asc
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-hadoop3.tgz.sha512
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-without-hadoop.tgz   (with props)
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-without-hadoop.tgz.asc
dev/spark/v3.1.3-rc3-bin/spark-3.1.3-bin-without-hadoop.tgz.sha512
dev/spark/v3.1.3-rc3-bin/spark-3.1.3.tgz   (with props)
dev/spark/v3.1.3-rc3-bin/spark-3.1.3.tgz.asc
dev/spark/v3.1.3-rc3-bin/spark-3.1.3.tgz.sha512

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

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

Added: dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz.asc
==
--- dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz.asc (added)
+++ dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz.asc Tue Feb  1 22:17:13 2022
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJGBAABCgAwFiEEyhnizCAbZSLToqBdxBq2dEJPw/sFAmH5sR8SHGhvbGRlbkBh
+cGFjaGUub3JnAAoJEMQatnRCT8P7xdsP/iSa4EDdX7WLWwv4dnJB7UhFAjey4Vvw
+5OygAu7C9BKaUwTvJ1ywmO82Qca6ze5doORpIGWfzj/cUkmZd3Qb/T5pU0gQkUAr
+7ovVT2DGClTdwgbfhMNEHW9nj5Y5hclKzeGxKaGT5qwOc5PDd9fKtgzIjQbuMGpy
+SLwcoikHpwUTjClWw9IMB+n3NQbOGTUpBZJDtcjUwe1BhAwOCL2B85eTj6XYA2LN
+SoARz1MwHyrH41Elh7ITdd1S9ynqbT00HqEPYMRDFN9DOzv2nWwobTdxpSmpLAJq
+Lz+wSDWJjS3x4spMJzNX8yvFAOlrPwiRGsRziNGm01XVd6vbDMTNj+ykaw9hkdnu
+xzjHmOtU/IKCc8YIOpobC00/tDrrgSktA44/7DPRC8UOreoxZkeDZWw0v942vns2
+YYFylq6zok9Fcqx5rLvDYiaQ1KLe4fXbdqXoqBYv2g0TKpZ55qdWtlc6In+Amui3
+YDAsAV1dHbPrM4C0JhZScWkHQLGq4ELnyt5LRRmOrkCE4+RpXw8EZEnSxgsxeDSn
+MJHtHg9QEm/HnuucjQn9Gko6nBqlj/wNjKFOA/0tF+o/1uYfDhNSdHTBdnf2/iaM
+uJ4Z27U06Ukp5aJ/Hi8iT2WJjHNZUViHbjAar3R83IlpDZ7S9bwjoDTJSEscilDM
+35YJ3kDXxEJp
+=pReb
+-END PGP SIGNATURE-

Added: dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz.sha512
==
--- dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz.sha512 (added)
+++ dev/spark/v3.1.3-rc3-bin/SparkR_3.1.3.tar.gz.sha512 Tue Feb  1 22:17:13 2022
@@ -0,0 +1,3 @@
+SparkR_3.1.3.tar.gz: 5DAF9715 DD2967CD A65922B1 B5E07284 E9F042D4 235D944A
+ A44E11A3 3BDE7F1B 703B0D0F BE8B5CAB F812E021 2FD7A806
+ EF944261 C497B696 363D1958 E7BA4754

Added: dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz.asc
==
--- dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz.asc (added)
+++ dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz.asc Tue Feb  1 22:17:13 2022
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJGBAABCgAwFiEEyhnizCAbZSLToqBdxBq2dEJPw/sFAmH5sSASHGhvbGRlbkBh
+cGFjaGUub3JnAAoJEMQatnRCT8P7jrQP/15vJX02cBXhu9lEvO6YBSvLX7y2QNMJ
+gkwrKs99km0kiinkhqfZYPQC9kRwT4W+psIP1+yplPMyOAfDCqzS9SAXJF2KsBhJ
+cfi4oKrK5R/6reBjq6bJfCOs68bHhND5u+GGm1zdkA7DHVQ9VUKMtENp6cvo4/xQ
+NU9TCP7YfLyKu1dY/N1csh0FyljpnoSbDgy8ZfO4q2dqcl/l2xQuFhYL7yE9MP35
+l/ReuWBfuAM8B8WQmVDTb1r74a1SBARyKj/u1PXyiKFxZEVYRniua/Di+uuAxz0A
+bT1RGG+2DrX+OuFBw2udSB0L6kLLXL7Oa3DBVakAahVFGDwHemLhclCVxdQHr11z
+yKRIExOR5GMy/80egx0R/X0AFgdatCQRebdDds3UuMeFCInRaBtcsxIPhFBm+D9u
+SRZft8NacQX1EZW35ZCEJQWNX+QOAbXoX+OoAvs8OZkN6V8wrDi2M+/H8d0HkjL+
+Ujts3vfnpc9F41B1/hR0NeR0LIQo0xjVMlEwab1HBy8j0nu4EX9Aks+nPAERTjM6
+wKdK4TUKUyr/jWw+q9A55ocBYGMr2kB0amfp6sPvzuD4CxQP3PoJiq1qryuWZmIG
+de1n4jp8BTP6LTq9yUidKc1NhissR6M7z9ZfCaNr6iVE4K+Ds/9U+dshmInreOx5
+lQqcn+AeHFjE
+=ihZ6
+-END PGP SIGNATURE-

Added: dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz.sha512
==
--- dev/spark/v3.1.3-rc3-bin/pyspark-3.1.3.tar.gz.sha512

[spark] 01/01: Preparing development version 3.1.4-SNAPSHOT

2022-02-01 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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

commit 7540421a2f26154be22852e876b9acfd004a85d3
Author: Holden Karau 
AuthorDate: Tue Feb 1 21:15:31 2022 +

Preparing development version 3.1.4-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 d42841c..db689ea 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.1.3
+Version: 3.1.4
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = "aut",
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 95309ee..8e2cded 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 80f6630..baf20e6 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index ead7a4f..1a9c791 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 232abd7..8a188a3 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 79068a6..d53f21d 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 2030f85..53b7312 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/tags

[spark] branch branch-3.1 updated (0a7eda3 -> 7540421)

2022-02-01 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


from 0a7eda3  [SPARK-38075][SQL][3.1] Fix `hasNext` in 
`HiveScriptTransformationExec`'s process output iterator
 add b8c0799  Preparing Spark release v3.1.3-rc3
 new 7540421  Preparing development version 3.1.4-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:

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



[spark] 01/01: Preparing Spark release v3.1.3-rc3

2022-02-01 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to tag v3.1.3-rc3
in repository https://gitbox.apache.org/repos/asf/spark.git

commit b8c0799a8cef22c56132d94033759c9f82b0cc86
Author: Holden Karau 
AuthorDate: Tue Feb 1 21:15:27 2022 +

Preparing Spark release v3.1.3-rc3
---
 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 db689ea..d42841c 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.1.4
+Version: 3.1.3
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = "aut",
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 8e2cded..95309ee 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index baf20e6..80f6630 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 1a9c791..ead7a4f 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 8a188a3..232abd7 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index d53f21d..79068a6 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 53b7312..2030f85 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/common

[spark] tag v3.1.3-rc3 created (now b8c0799)

2022-02-01 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a change to tag v3.1.3-rc3
in repository https://gitbox.apache.org/repos/asf/spark.git.


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

 new b8c0799  Preparing Spark release v3.1.3-rc3

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



svn commit: r52259 - in /dev/spark/v3.1.3-rc2-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _site/api/java/org/apache/parqu

2022-01-25 Thread holden
Author: holden
Date: Tue Jan 25 08:02:31 2022
New Revision: 52259

Log:
Apache Spark v3.1.3-rc2 docs


[This commit notification would consist of 2264 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: r52233 - /dev/spark/v3.1.3-rc2-bin/

2022-01-23 Thread holden
Author: holden
Date: Sun Jan 23 20:03:24 2022
New Revision: 52233

Log:
Apache Spark v3.1.3-rc2

Added:
dev/spark/v3.1.3-rc2-bin/
dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz   (with props)
dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz.asc
dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz.sha512
dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz   (with props)
dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz.asc
dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz.sha512
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-hadoop2.tgz   (with props)
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-hadoop2.tgz.asc
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-hadoop2.tgz.sha512
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-hadoop3.tgz   (with props)
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-hadoop3.tgz.asc
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-hadoop3.tgz.sha512
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-without-hadoop.tgz   (with props)
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-without-hadoop.tgz.asc
dev/spark/v3.1.3-rc2-bin/spark-3.1.3-bin-without-hadoop.tgz.sha512
dev/spark/v3.1.3-rc2-bin/spark-3.1.3.tgz   (with props)
dev/spark/v3.1.3-rc2-bin/spark-3.1.3.tgz.asc
dev/spark/v3.1.3-rc2-bin/spark-3.1.3.tgz.sha512

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

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

Added: dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz.asc
==
--- dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz.asc (added)
+++ dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz.asc Sun Jan 23 20:03:24 2022
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJFBAABCgAwFiEEyhnizCAbZSLToqBdxBq2dEJPw/sFAmHttEsSHGhvbGRlbkBh
+cGFjaGUub3JnAAoJEMQatnRCT8P7U+YP9A7mD4KFkJxrGgVFcjWn0Ll5aG0aDpPP
+jXb3hFeEEkUALFLzqS6Nnms03cgRNIysIl8RU5JPVkxmQa7FWc3wMkqrzGPKusDP
+v842qO6uGLxYqg2EM9PKpz0bV+Lzo2/CqJ8hZKfiJGoAuLv2upPoz80iuV+Ey29a
+XKa4AwNiQFQAzMHfG/ydHx2to9nxx3w2Y2PNgsYKrTT2zN/X1i5bCxinCasX5Hm1
+gP8CyfQwTZJnmzg7U1iAnSoOIsf6vG63Vfnb3aKP5+Ee96ny1vSkT29YJVPaFD+r
+/ItbDZbCI2rhm0AAWZGF4O4BEP7Qq5HAnaohwXA9P0A07flNT7rdL/bTuGS6+8Sq
+vRlqpMvS3N+kxH/c1l36hvoZ1V/uVv8v85ygaWZ2ufIFCPaApufeI4Oc8Xf+ujVF
+6zFKH0eNtDVXf0sL0Cs9jezBtA1+n0a3z3cigzw6qVy3nZF69SSzbEEEOHmToLgB
+q243s4Mk4B3rOZThW5crHSYFLOS7YraVf6Ocv7jVOl47CN6jCS47Q5CkIS3spW1p
+6mLixT47QdgGBDoqUvwW9afWjsfjeRYbUtXFO/loPQteIRnQtVWOGhIxEEGkXZYU
+Yb/fnvYpwvlJT1x28OQ1FXts0uuUgS7Cfj891t1mwQm5f0uibgt0bgBxa7ZblkOh
+M/eBzY91vOE=
+=jyno
+-END PGP SIGNATURE-

Added: dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz.sha512
==
--- dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz.sha512 (added)
+++ dev/spark/v3.1.3-rc2-bin/SparkR_3.1.3.tar.gz.sha512 Sun Jan 23 20:03:24 2022
@@ -0,0 +1,3 @@
+SparkR_3.1.3.tar.gz: 2520DA4C 77648843 B0F0F4AF 81930F82 0DC3B8A6 DD474285
+ 28FC3DD4 7CDF7EFF 6CFD0982 4AC4D12B 0A8A0633 968EF8E1
+ 1B92F48B 7D87DC3B 39BF18DB 67645A7D

Added: dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz.asc
==
--- dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz.asc (added)
+++ dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz.asc Sun Jan 23 20:03:24 2022
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJGBAABCgAwFiEEyhnizCAbZSLToqBdxBq2dEJPw/sFAmHttE0SHGhvbGRlbkBh
+cGFjaGUub3JnAAoJEMQatnRCT8P7kvEP/inJR1yX1EAGXPFkqa89AYd4bf4uQw78
+pYPGMPxxGW7/HjwuIlzxiFcMUwQl/r0n9iinuF9Hs/z0nzwMp2JzL9fmiZDVNV+8
++wyt7kke3VvXBJzzO7XYJ+bQw3xPTbdmmkfdgL+OPalNVOXueamf+0siANdXaFRL
+XNqWWMsp2ujXk+PiQulipumJ4hG3p/LkQAvTPfgpgvr1yqh25TZQUY+o6DB1RN+b
+mlkTT47oPWSixS9OdfQZbVBq5YjFa4iSr77I9Bv9BxJDJE13ASHoJ81+yrK3g2eF
+a+1UTTTxk9tVL0TAl14TIeC2T0wzDQHcHaSvNYfykeSIbn+0Ou74diWrSCArWRPE
+wyThcUfwE971HdB8+l/5qh3D/DNCAQGELQSw+cw8eDG9lpMZqwYzGCSKk2W7xTRi
+sEbj2UKd8wbW+JIb0XYrsAukKkJiB3Dv30h4o1cFaSKTXmiVOR/SY7ixs8AEG48m
+EmpHN766yS9u6uu/AOJd7vQ2wzB2egNMt8z/z/20jUOt3D4rNde6jN0DcXbhbCEA
+wUxvqjqD8Cwm5sVLIBd+nNYKv6FNB5pLDOeakPZJri6EGhgGiWZD3J53mSf0QFLX
+dLBmDchZqVE9rF1KbskWysNrWcxOJL41BYAbtW8YrdibjN9gSKIL+h95A6CbkDkL
+mqeVOFxNrskc
+=RXYl
+-END PGP SIGNATURE-

Added: dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz.sha512
==
--- dev/spark/v3.1.3-rc2-bin/pyspark-3.1.3.tar.gz.sha512

[spark] branch branch-3.1 updated (830d5b6 -> 91db9a3)

2022-01-13 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


from 830d5b6  [SPARK-37860][UI] Fix taskindex in the stage page task event 
timeline
 add fd2d694  Preparing Spark release v3.1.3-rc2
 new 91db9a3  Preparing development version 3.1.4-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:

-
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.1.4-SNAPSHOT

2022-01-13 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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

commit 91db9a36a9ed74845908f14d21227d5267591653
Author: Holden Karau 
AuthorDate: Thu Jan 13 17:42:17 2022 +

Preparing development version 3.1.4-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 d42841c..db689ea 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.1.3
+Version: 3.1.4
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = "aut",
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 95309ee..8e2cded 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 80f6630..baf20e6 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index ead7a4f..1a9c791 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 232abd7..8a188a3 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 79068a6..d53f21d 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 2030f85..53b7312 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/tags

[spark] 01/01: Preparing Spark release v3.1.3-rc2

2022-01-13 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to tag v3.1.3-rc2
in repository https://gitbox.apache.org/repos/asf/spark.git

commit fd2d694d783637e496a72166445b1853625c4cbd
Author: Holden Karau 
AuthorDate: Thu Jan 13 17:42:13 2022 +

Preparing Spark release v3.1.3-rc2
---
 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 db689ea..d42841c 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.1.4
+Version: 3.1.3
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = "aut",
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 8e2cded..95309ee 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index baf20e6..80f6630 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 1a9c791..ead7a4f 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 8a188a3..232abd7 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index d53f21d..79068a6 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.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 53b7312..2030f85 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.4-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/common

[spark] tag v3.1.3-rc2 created (now fd2d694)

2022-01-13 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a change to tag v3.1.3-rc2
in repository https://gitbox.apache.org/repos/asf/spark.git.


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

 new fd2d694  Preparing Spark release v3.1.3-rc2

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



svn commit: r52050 - /dev/spark/v3.1.3-rc1-bin/

2022-01-12 Thread holden
Author: holden
Date: Wed Jan 12 22:19:54 2022
New Revision: 52050

Log:
Apache Spark v3.1.3-rc1

Added:
dev/spark/v3.1.3-rc1-bin/
dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz   (with props)
dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz.asc
dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz.sha512
dev/spark/v3.1.3-rc1-bin/pyspark-3.1.3.tar.gz   (with props)
dev/spark/v3.1.3-rc1-bin/pyspark-3.1.3.tar.gz.asc
dev/spark/v3.1.3-rc1-bin/pyspark-3.1.3.tar.gz.sha512
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-hadoop2.7.tgz   (with props)
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-hadoop2.7.tgz.asc
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-hadoop2.7.tgz.sha512
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-hadoop3.2.tgz   (with props)
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-hadoop3.2.tgz.asc
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-hadoop3.2.tgz.sha512
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-without-hadoop.tgz   (with props)
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-without-hadoop.tgz.asc
dev/spark/v3.1.3-rc1-bin/spark-3.1.3-bin-without-hadoop.tgz.sha512
dev/spark/v3.1.3-rc1-bin/spark-3.1.3.tgz   (with props)
dev/spark/v3.1.3-rc1-bin/spark-3.1.3.tgz.asc
dev/spark/v3.1.3-rc1-bin/spark-3.1.3.tgz.sha512

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

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

Added: dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz.asc
==
--- dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz.asc (added)
+++ dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz.asc Wed Jan 12 22:19:54 2022
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJGBAABCgAwFiEEyhnizCAbZSLToqBdxBq2dEJPw/sFAmHfSr0SHGhvbGRlbkBh
+cGFjaGUub3JnAAoJEMQatnRCT8P73cgP/j9op+sFsqEzSUZ+VYTOA+ttS3wIDNnw
+wvzuaeJzpYmImJj51v6zACD48z5fYY3zjlK0yNwlw0Zho0L0aofjOVaG0eS6RCC3
+sk7dcLj31vrPkt75aSswe2tLtWlVZ94glua1FwuIWckJW9hiogL93Zpm3RsoDxWY
+/r7JluXJ1cyab4o8mXsKLczzC3GULST609Q6h/CNHK627SCqsfM6dBWG+zFxqEtT
+NaoYHvSGC0CWlfVYlbGfCK7J9yb8+KcaxBOCm38axveXjJF1NOb4ptfdRh54RRc4
+Ekrk6WxyGDMYqpfjY7ICfGNUfynX5MNk79OkccZU0OneikVDFpy+do6HK0G2W09r
+ViG2C+ccW5UjzoiNAqK+4p7NKRHtfH6RW9sCe+V6HYuZOGBRwlwSofsGxy4Mk1Z/
+ngWpVSrSs7r1cSY7js4uXCWSycLTx+O7e9u9nU1/cvWtiXTUIaFrDQ2pahP1ug9E
+i2hFWq87XfsNNkLmrjpmcmxedBNiDhkix7J3q2wiawIWo8V82puPODbnADUlIDeg
+E/vuW3wfOHeVc3qwDehXVstWEzdwmBwNxfVEjLu5q0x8enERbj3Ud9JyqjDXwA34
+59eOXtPpt1Ebc4Xmv6kDVZi1MNicF34r9JklgkWOeplqP9BJpUr56IvZDbVw1O+B
+Ca/cRHiS7BNt
+=RALG
+-END PGP SIGNATURE-

Added: dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz.sha512
==
--- dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz.sha512 (added)
+++ dev/spark/v3.1.3-rc1-bin/SparkR_3.1.3.tar.gz.sha512 Wed Jan 12 22:19:54 2022
@@ -0,0 +1,3 @@
+SparkR_3.1.3.tar.gz: 60FA6C02 865717E3 DDF94831 5B8A2C24 B22EAAF5 2D8D35D2
+ A0F569A2 50DA6021 CBCE131F C336011B A5D2A8E5 74F5A85F
+ 653AC05A D3684B0C CB663932 B1E950E2

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

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

Added: dev/spark/v3.1.3-rc1-bin/pyspark-3.1.3.tar.gz.asc
==
--- dev/spark/v3.1.3-rc1-bin/pyspark-3.1.3.tar.gz.asc (added)
+++ dev/spark/v3.1.3-rc1-bin/pyspark-3.1.3.tar.gz.asc Wed Jan 12 22:19:54 2022
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+
+iQJGBAABCgAwFiEEyhnizCAbZSLToqBdxBq2dEJPw/sFAmHfSr4SHGhvbGRlbkBh
+cGFjaGUub3JnAAoJEMQatnRCT8P73WwP/3hS1LoVx4LD9sC15cagBRvm95AgySzO
+3Quf6791doVk8tsyd65WmVITmF1Yv66rhOgW29IpoGq7mJmjxIkPwmW8qbhCHi8S
+wIs88/DRqF+bL7M9+jS9c0quE7/5gE3PgRIhKnunUQfnekj48k0mQM0Za6lq+vhN
+E/nTvQvdxznFHoN1j9wCmYRyTB7h4MOGClDIutMvX4yERmfwTgicUxfTlhxuaETd
+ZS/v8wvcZU91WHntyxAkwwWTHG2NO03vH5qWVnLePY6IrZCa2mCM3EpxLXuB3HwX
+jWLQx08Mb39yhdDxO8zrlsj6W3C7rkqN87bKU7m9oG7N/5lDEuYlMTu6ZU6WC2Af
++G5ChOfxVSa6s1JiFK2h8DzszksOHqnZGLFcDuBKsda5NU/Bgrwd2lKrXTaurXvE
+NtT0Qa3LTNs0ivevI1pohWqc08iV20mh7ij8RzQRNP+vilxLI9kj/hwUn8EZpbif
+gHuZUFw/UxhEO+fenS9gkYlhvIxECfioJ7ZoEygPx1XeenBJY2+fbcrlPWafdA1o
+FK7wzdf/dSko7Z4e7TvH74xKn1iNG9xY6dYpH50MI+flzkj4eL2U2sGQ7cwdIA/A
+pAsKjZXTknl3il+S5VT6tiGPZd270asF/EcSQJCEv4UPSa/tQt/2Ac1VpiMZMNlY
+Vut5eqb+vmxB
+=kHOB
+-END PGP SIGNATURE-

Added: dev/spark/v3.1.3-rc1-bin/pyspark-3.1.3.tar.gz.sha512
==
--- dev/spark/v3.1.3-rc1-bin/pyspark-3.1.3

[spark] 01/01: Preparing development version 3.1.4-SNAPSHOT

2022-01-10 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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

commit 1a275396da176d9b0436066f0173ec24bc128ab9
Author: Holden Karau 
AuthorDate: Mon Jan 10 21:23:44 2022 +

Preparing development version 3.1.4-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 d42841c..db689ea 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.1.3
+Version: 3.1.4
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = "aut",
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 95309ee..8e2cded 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 80f6630..baf20e6 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index ead7a4f..1a9c791 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 232abd7..8a188a3 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 79068a6..d53f21d 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.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 2030f85..53b7312 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3
+3.1.4-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/tags

[spark] branch branch-3.1 updated (94a69ff -> 1a27539)

2022-01-10 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


from 94a69ff  Revert "[SPARK-37779][SQL] Make ColumnarToRowExec plan 
canonicalizable after (de)serialization"
 add df89eb2  Preparing Spark release v3.1.3-rc1
 new 1a27539  Preparing development version 3.1.4-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.1.3-rc1 created (now df89eb2)

2022-01-10 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


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

 new df89eb2  Preparing Spark release v3.1.3-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] 01/01: Preparing Spark release v3.1.3-rc1

2022-01-10 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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

commit df89eb2d748c99665256d3ef297fef774db1014a
Author: Holden Karau 
AuthorDate: Mon Jan 10 21:23:40 2022 +

Preparing Spark release v3.1.3-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 bf75946..95309ee 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3-SNAPSHOT
+3.1.3
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 777477b..80f6630 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 2233100..ead7a4f 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.1.3-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index aa5b2d4..232abd7 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.1.3-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index d3cc674..79068a6 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.1.3-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index c54d485..2030f85 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/common/tags/pom.xml
index a02b306..798c88e 100644
--- a/common/tags/pom.xml
+++ b/common/tags/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml
index 64ee6ba..bd829c0 100644
--- a/common/unsafe/pom.xml
+++ b/common/unsafe/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.1.3-SNAPSHOT
+3.1.3
 ../../pom.xml
   
 
diff --git

[spark] branch master updated: [SPARK-36058][K8S] Add support for statefulset APIs in K8s

2021-08-25 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 ff3f3c4  [SPARK-36058][K8S] Add support for statefulset APIs in K8s
ff3f3c4 is described below

commit ff3f3c45668364da9bd10992791b5ee9a46fea21
Author: Holden Karau 
AuthorDate: Wed Aug 25 17:38:57 2021 -0700

[SPARK-36058][K8S] Add support for statefulset APIs in K8s

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

Generalize the pod allocator and add support for statefulsets.

### Why are the changes needed?

Allocating individual pods in Spark can be not ideal for some clusters and 
using higher level operators like statefulsets and replicasets can be useful.

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

Yes new config options.

### How was this patch tested?

Completed: New unit & basic integration test
PV integration tests

Closes #33508 from 
holdenk/SPARK-36058-support-replicasets-or-job-api-like-things.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../executor/CoarseGrainedExecutorBackend.scala|   2 +-
 .../scala/org/apache/spark/storage/DiskStore.scala |   9 +-
 .../apache/spark/examples/MiniReadWriteTest.scala  | 139 +
 .../scala/org/apache/spark/deploy/k8s/Config.scala |  10 +
 .../org/apache/spark/deploy/k8s/Constants.scala|   1 +
 .../k8s/features/BasicExecutorFeatureStep.scala|  15 +-
 .../cluster/k8s/AbstractPodsAllocator.scala|  59 ++
 .../cluster/k8s/ExecutorPodsAllocator.scala|  18 +-
 .../cluster/k8s/ExecutorPodsSnapshot.scala |  11 +-
 .../cluster/k8s/KubernetesClusterManager.scala |  38 +++-
 .../cluster/k8s/KubernetesClusterMessage.scala |  21 ++
 .../k8s/KubernetesClusterSchedulerBackend.scala|  41 +++-
 .../cluster/k8s/KubernetesExecutorBackend.scala| 228 +
 .../cluster/k8s/StatefulsetPodsAllocator.scala | 201 ++
 .../features/BasicExecutorFeatureStepSuite.scala   |   6 +-
 .../cluster/k8s/ExecutorPodsAllocatorSuite.scala   |  31 ++-
 .../k8s/KubernetesClusterManagerSuite.scala|  58 ++
 .../KubernetesClusterSchedulerBackendSuite.scala   |  13 +-
 .../cluster/k8s/StatefulsetAllocatorSuite.scala| 153 ++
 .../src/main/dockerfiles/spark/entrypoint.sh   |   3 +-
 .../k8s/integrationtest/BasicTestsSuite.scala  |  17 ++
 .../k8s/integrationtest/KubernetesSuite.scala  |  33 ++-
 .../deploy/k8s/integrationtest/PVTestsSuite.scala  |  88 +++-
 23 files changed, 1148 insertions(+), 47 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 76d01f8..c87e61a 100644
--- 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -61,7 +61,7 @@ private[spark] class CoarseGrainedExecutorBackend(
 
   private implicit val formats = DefaultFormats
 
-  private[executor] val stopping = new AtomicBoolean(false)
+  private[spark] val stopping = new AtomicBoolean(false)
   var executor: Executor = null
   @volatile var driver: Option[RpcEndpointRef] = None
 
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala 
b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
index 7269913..f0334c5 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
@@ -59,7 +59,14 @@ private[spark] class DiskStore(
*/
   def put(blockId: BlockId)(writeFunc: WritableByteChannel => Unit): Unit = {
 if (contains(blockId)) {
-  throw new IllegalStateException(s"Block $blockId is already present in 
the disk store")
+  logWarning(s"Block $blockId is already present in the disk store")
+  try {
+diskManager.getFile(blockId).delete()
+  } catch {
+case e: Exception =>
+  throw new IllegalStateException(
+s"Block $blockId is already present in the disk store and could 
not delete it $e")
+  }
 }
 logDebug(s"Attempting to put block $blockId")
 val startTimeNs = System.nanoTime()
diff --git 
a/examples/src/main/scala/org/apache/spark/examples/MiniReadWriteTest.scala 
b/examples/src/main/scala/org/apache/spark/examples/MiniReadWriteTest.scala
new file mode 100644
index 000..5a74e1c
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/MiniReadWriteTest.scala
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor li

[spark] branch master updated: [SPARK-34309][BUILD][CORE][SQL][K8S] Use Caffeine instead of Guava Cache

2021-08-04 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 01cf6f4  [SPARK-34309][BUILD][CORE][SQL][K8S] Use Caffeine instead of 
Guava Cache
01cf6f4 is described below

commit 01cf6f4c6b2a593a2a8717fd2cda13725424120e
Author: yangjie01 
AuthorDate: Wed Aug 4 12:01:44 2021 -0700

[SPARK-34309][BUILD][CORE][SQL][K8S] Use Caffeine instead of Guava Cache

### What changes were proposed in this pull request?
There are 3 ways to use Guava cache in spark code:

1. `Loadingcache` is the main way to use Guava cache in spark code and the 
key usages are as follows:
  a. `LoadingCache` with `maximumsize` data eviction policy, such as 
`appCache` in `ApplicationCache`, `cache` in `Codegenerator`
  b. `LoadingCache` with `maximumWeight` data eviction policy, such as 
`shuffleIndexCache` in `ExternalShuffleBlockResolver`
  c. `LoadingCache` with 'expireAfterWrite' data eviction policy, such as 
`tableRelationCache` in `SessionCatalog`
2. `ManualCache` is another way to use Guava cache in spark code and the 
key usage is `cache` in `SharedInMemoryCache`, it use to caches partition file 
statuses in memory

3. The last use way is `hadoopJobMetadata` in `SparkEnv`, it uses Guava 
Cache to build a `soft-reference map`.

The goal of this pr is use `Caffeine` instead of `Guava Cache` because 
`Caffeine` is faster than `Guava Cache` from benchmarks, the main changes as 
follows:

1. Add `Caffeine` deps to maven `pom.xml`

2. Use `Caffeine` instead of Guava `LoadingCache`, `ManualCache` and 
soft-reference map in `SparkEnv`

3. Add `LocalCacheBenchmark` to compare performance of `Loadingcache` 
between `Guava Cache` and `Caffeine`

### Why are the changes needed?
`Caffeine` is faster than `Guava Cache` from benchmarks

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

### How was this patch tested?

- Pass the Jenkins or GitHub Action
- Add `LocalCacheBenchmark` to compare performance of `Loadingcache` 
between `Guava Cache` and `Caffeine`

Closes #31517 from LuciferYang/guava-cache-to-caffeine.

Authored-by: yangjie01 
Signed-off-by: Holden Karau 
---
 common/network-shuffle/pom.xml |  4 +
 .../shuffle/ExternalShuffleBlockResolver.java  | 32 +++-
 .../network/shuffle/RemoteBlockPushResolver.java   | 24 +++---
 .../LocalCacheBenchmark-jdk11-results.txt  | 12 +++
 core/benchmarks/LocalCacheBenchmark-results.txt| 12 +++
 core/pom.xml   |  4 +
 .../src/main/scala/org/apache/spark/SparkEnv.scala |  4 +-
 .../spark/deploy/history/ApplicationCache.scala| 37 +
 .../apache/spark/rdd/ReliableCheckpointRDD.scala   | 24 +++---
 .../org/apache/spark/storage/BlockManager.scala|  4 +-
 .../org/apache/spark/storage/BlockManagerId.scala  | 15 ++--
 .../spark/storage/BlockManagerMasterEndpoint.scala |  4 +-
 .../main/scala/org/apache/spark/util/Utils.scala   | 19 +++--
 .../org/apache/spark/LocalCacheBenchmark.scala | 93 ++
 .../org/apache/spark/executor/ExecutorSuite.scala  | 18 ++---
 dev/deps/spark-deps-hadoop-2.7-hive-2.3|  3 +
 dev/deps/spark-deps-hadoop-3.2-hive-2.3|  3 +
 pom.xml|  6 ++
 resource-managers/kubernetes/core/pom.xml  |  5 ++
 .../cluster/k8s/ExecutorPodsLifecycleManager.scala |  9 ++-
 sql/catalyst/pom.xml   |  4 +
 .../sql/catalyst/catalog/SessionCatalog.scala  |  9 +--
 .../expressions/SubExprEvaluationRuntime.scala | 21 ++---
 .../expressions/codegen/CodeGenerator.scala|  6 +-
 .../catalyst/util/DateTimeFormatterHelper.scala|  4 +-
 ...CodeGeneratorWithInterpretedFallbackSuite.scala |  4 +-
 .../SubExprEvaluationRuntimeSuite.scala| 20 ++---
 sql/core/pom.xml   |  5 +-
 .../execution/datasources/FileStatusCache.scala| 13 ++-
 .../spark/sql/execution/metric/SQLMetrics.scala|  4 +-
 30 files changed, 283 insertions(+), 139 deletions(-)

diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index d3d78f2..1b78182 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -59,6 +59,10 @@
   provided
 
 
+  com.github.ben-manes.caffeine
+  caffeine
+
+
   com.google.guava
   guava
 
diff --git 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java
index 73d4e6c..650f33e 100644
--- 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle

[spark] branch master updated: [SPARK-35956][K8S] Support auto assigning labels to decommissioning pods

2021-07-23 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 bee2799  [SPARK-35956][K8S] Support auto assigning labels to 
decommissioning pods
bee2799 is described below

commit bee279997f2115af6b15e3dbb7433dccef7f14af
Author: Holden Karau 
AuthorDate: Fri Jul 23 15:21:38 2021 -0700

[SPARK-35956][K8S] Support auto assigning labels to decommissioning pods

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

Add a new configuration flag to allow Spark to provide hints to the 
scheduler when we are decommissioning or exiting a pod that this pod will have 
the least impact for a pre-emption event.

### Why are the changes needed?

Kubernetes added the concepts of pod disruption budgets (which can have 
selectors based on labels) as well pod deletion for providing hints to the 
scheduler as to what we would prefer to have pre-empted.

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

New configuration flag

### How was this patch tested?

The deletion unit test was extended.

Closes #33270 from 
holdenk/SPARK-35956-support-auto-assigning-labels-to-decommissioning-pods.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 docs/running-on-kubernetes.md  | 28 +---
 .../scala/org/apache/spark/deploy/k8s/Config.scala | 18 
 .../k8s/KubernetesClusterSchedulerBackend.scala| 50 +-
 .../KubernetesClusterSchedulerBackendSuite.scala   | 47 +++-
 .../k8s/integrationtest/DecommissionSuite.scala| 40 ++---
 5 files changed, 168 insertions(+), 15 deletions(-)

diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md
index 09f7d2ab..b30d61d 100644
--- a/docs/running-on-kubernetes.md
+++ b/docs/running-on-kubernetes.md
@@ -8,9 +8,9 @@ license: |
   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.
@@ -422,7 +422,7 @@ Your Kubernetes config file typically lives under 
`.kube/config` in your home di
 
 ### Contexts
 
-Kubernetes configuration files can contain multiple contexts that allow for 
switching between different clusters and/or user identities.  By default Spark 
on Kubernetes will use your current context (which can be checked by running 
`kubectl config current-context`) when doing the initial auto-configuration of 
the Kubernetes client.  
+Kubernetes configuration files can contain multiple contexts that allow for 
switching between different clusters and/or user identities.  By default Spark 
on Kubernetes will use your current context (which can be checked by running 
`kubectl config current-context`) when doing the initial auto-configuration of 
the Kubernetes client.
 
 In order to use an alternative context users can specify the desired context 
via the Spark configuration property `spark.kubernetes.context` e.g. 
`spark.kubernetes.context=minikube`.
 
@@ -1038,7 +1038,7 @@ See the [configuration page](configuration.html) for 
information on Spark config
spark.kubernetes.executor.secrets.ENV_VAR=spark-secret:key.
   
   2.4.0
-   
+
 
   
spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.path
   (none)
@@ -1270,7 +1270,7 @@ See the [configuration page](configuration.html) for 
information on Spark config
   
   3.0.0
 
-  
+
   spark.kubernetes.appKillPodDeletionGracePeriod
   (none)
   
@@ -1288,6 +1288,24 @@ See the [configuration page](configuration.html) for 
information on Spark config
   
   3.0.0
 
+
+  spark.kubernetes.executor.decommmissionLabel
+  (none)
+  
+Label to be applied to pods which are exiting or being decommissioned. 
Intended for use
+with pod disruption budgets, deletion costs, and similar.
+  
+  3.3.0
+
+
+  spark.kubernetes.executor.decommmissionLabelValue
+  (none)
+  
+Value to be applied with the label when
+spark.kubernetes.executor.decommmissionLabel is enabled.
+  
+  3.3.0
+
 
 
  Pod template properties
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
index 49c0a42..33370b7 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
+++ 

[spark] branch master updated: [SPARK-34764][CORE][K8S][UI] Propagate reason for exec loss to Web UI

2021-05-13 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 160b3be  [SPARK-34764][CORE][K8S][UI] Propagate reason for exec loss 
to Web UI
160b3be is described below

commit 160b3bee71ac5c4b5587fa16c8eef753e9a4ad91
Author: Holden Karau 
AuthorDate: Thu May 13 16:02:31 2021 -0700

[SPARK-34764][CORE][K8S][UI] Propagate reason for exec loss to Web UI

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

Adds the exec loss reason to the Spark web UI & in doing so also fix the 
Kube integration to pass exec loss reason into core.

UI change:


![image](https://user-images.githubusercontent.com/59893/117045762-b975ba80-acc4-11eb-9679-8edab3cfadc2.png)

### Why are the changes needed?

Debugging Spark jobs is *hard*, making it clearer why executors have exited 
could help.

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

Yes a new column on the executor page.

### How was this patch tested?

K8s unit test updated to validate exec loss reasons are passed through 
regardless of exec alive state, manual testing to validate the UI.

Closes #32436 from holdenk/SPARK-34764-propegate-reason-for-exec-loss.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../spark/ui/static/executorspage-template.html|  1 +
 .../org/apache/spark/ui/static/executorspage.js| 15 +-
 .../cluster/k8s/ExecutorPodsLifecycleManager.scala | 34 --
 .../k8s/KubernetesClusterSchedulerBackend.scala|  4 +--
 .../k8s/ExecutorPodsLifecycleManagerSuite.scala| 21 -
 .../KubernetesClusterSchedulerBackendSuite.scala   |  4 +--
 6 files changed, 63 insertions(+), 16 deletions(-)

diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html
 
b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html
index be6d7bc..37d56a0 100644
--- 
a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html
+++ 
b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html
@@ -128,6 +128,7 @@ limitations under the License.
   Shuffle Write
   Logs
   Thread Dump
+  Exec Loss Reason
 
 
 
diff --git 
a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js 
b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
index 2055c8f..5cc2868 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
@@ -31,6 +31,14 @@ function getThreadDumpEnabled() {
   return threadDumpEnabled;
 }
 
+function formatLossReason(removeReason, type, row) {
+if (removeReason) {
+   return removeReason
+} else {
+   return ""
+}
+}
+
 function formatStatus(status, type, row) {
   if (row.isExcluded) {
 return "Excluded";
@@ -132,7 +140,7 @@ function totalDurationColor(totalGCTime, totalDuration) {
 }
 
 var sumOptionalColumns = [3, 4];
-var execOptionalColumns = [5, 6, 7, 8, 9, 10, 13, 14];
+var execOptionalColumns = [5, 6, 7, 8, 9, 10, 13, 14, 15];
 var execDataTable;
 var sumDataTable;
 
@@ -543,6 +551,10 @@ $(document).ready(function () {
   data: 'id', render: function (data, type) {
 return type === 'display' ? ("Thread Dump" ) : data;
   }
+},
+{
+data: 'removeReason',
+render: formatLossReason
 }
   ],
   "order": [[0, "asc"]],
@@ -709,6 +721,7 @@ $(document).ready(function () {
   " Peak Pool Memory 
Direct / Mapped" +
   " 
Resources" +
   " Resource Profile Id" +
+  " Exec Loss Reason" +
   "");
 
 reselectCheckboxesBasedOnTaskTableState();
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala
index 67e8f7e..e255de4 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala
@@ -221,10 +221,17 @@ private[spark] class ExecutorPodsLifecycleManager(
 val pod = podState.pod
 val reason = Option(pod.getStatus.getReason)
 val message = Option(pod.getStatus.getMessage)
+val explained = describeExitCode(

[spark] branch branch-3.1 updated: [SPARK-34361][K8S] In case of downscaling avoid killing of executors already known by the scheduler backend in the pod allocator

2021-03-02 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 28f2d51  [SPARK-34361][K8S] In case of downscaling avoid killing of 
executors already known by the scheduler backend in the pod allocator
28f2d51 is described below

commit 28f2d51e5950a8b99a37814c9396b23315880fe5
Author: “attilapiros” 
AuthorDate: Tue Mar 2 16:58:29 2021 -0800

[SPARK-34361][K8S] In case of downscaling avoid killing of executors 
already known by the scheduler backend in the pod allocator

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

This PR modifies the POD allocator to use the scheduler backend to get the 
known executors and remove those from the pending and newly created list.

This is different from the normal `ExecutorAllocationManager` requested 
killing of executors where the  `spark.dynamicAllocation.executorIdleTimeout` 
is used.
In this case POD allocator kills the executors which  should be only 
responsible for terminating not satisfied POD allocations (new requests where 
no POD state is received yet and PODs in pending state).

### Why are the changes needed?

Because there is race between executor POD allocator and cluster scheduler 
backend.
Running several experiment during downscaling we experienced a lot of 
killed fresh executors wich has already running task on them.

The pattern in the log was the following (see executor 312 and TID 2079):

```
21/02/01 15:12:03 INFO ExecutorMonitor: New executor 312 has registered 
(new total is 138)
...
21/02/01 15:12:03 INFO TaskSetManager: Starting task 247.0 in stage 4.0 
(TID 2079, 100.100.18.138, executor 312, partition 247, PROCESS_LOCAL, 8777 
bytes)
21/02/01 15:12:03 INFO ExecutorPodsAllocator: Deleting 3 excess pod 
requests (408,312,307).
...
21/02/01 15:12:04 ERROR TaskSchedulerImpl: Lost executor 312 on 
100.100.18.138: The executor with id 312 was deleted by a user or the framework.
21/02/01 15:12:04 INFO TaskSetManager: Task 2079 failed because while it 
was being computed, its executor exited for a reason unrelated to the task. Not 
counting this failure towards the maximum number of failures for the task.
```

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

No.

### How was this patch tested?

 Manually

With this change there was no executor lost with running task on it.

# With unit test

A new test is added and existing test is modified to check these cases.

Closes #31513 from attilapiros/SPARK-34361.

Authored-by: “attilapiros” 
Signed-off-by: Holden Karau 
(cherry picked from commit 6c5322de6176726955b4bc941f92ecaa54a7f539)
Signed-off-by: Holden Karau 
---
 .../scala/org/apache/spark/deploy/k8s/Config.scala |   3 +-
 .../cluster/k8s/ExecutorPodsAllocator.scala|  66 ++---
 .../k8s/KubernetesClusterSchedulerBackend.scala|   2 +-
 .../cluster/k8s/ExecutorPodsAllocatorSuite.scala   | 155 -
 .../KubernetesClusterSchedulerBackendSuite.scala   |   2 +-
 5 files changed, 204 insertions(+), 24 deletions(-)

diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
index 68dcef1..bec29a9 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
@@ -237,7 +237,8 @@ private[spark] object Config extends Logging {
 
   val KUBERNETES_ALLOCATION_EXECUTOR_TIMEOUT =
 ConfigBuilder("spark.kubernetes.allocation.executor.timeout")
-  .doc("Time to wait before considering a pending executor timedout.")
+  .doc("Time to wait before a newly created executor POD request, which 
does not reached " +
+"the POD pending state yet, considered timedout and will be deleted.")
   .version("3.1.0")
   .timeConf(TimeUnit.MILLISECONDS)
   .checkValue(value => value > 0, "Allocation executor timeout must be a 
positive time value.")
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
index eb35de8..5fc81a6 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
@@ -18,7 +18,7 @@ package 

[spark] branch master updated: [SPARK-34361][K8S] In case of downscaling avoid killing of executors already known by the scheduler backend in the pod allocator

2021-03-02 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 6c5322d  [SPARK-34361][K8S] In case of downscaling avoid killing of 
executors already known by the scheduler backend in the pod allocator
6c5322d is described below

commit 6c5322de6176726955b4bc941f92ecaa54a7f539
Author: “attilapiros” 
AuthorDate: Tue Mar 2 16:58:29 2021 -0800

[SPARK-34361][K8S] In case of downscaling avoid killing of executors 
already known by the scheduler backend in the pod allocator

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

This PR modifies the POD allocator to use the scheduler backend to get the 
known executors and remove those from the pending and newly created list.

This is different from the normal `ExecutorAllocationManager` requested 
killing of executors where the  `spark.dynamicAllocation.executorIdleTimeout` 
is used.
In this case POD allocator kills the executors which  should be only 
responsible for terminating not satisfied POD allocations (new requests where 
no POD state is received yet and PODs in pending state).

### Why are the changes needed?

Because there is race between executor POD allocator and cluster scheduler 
backend.
Running several experiment during downscaling we experienced a lot of 
killed fresh executors wich has already running task on them.

The pattern in the log was the following (see executor 312 and TID 2079):

```
21/02/01 15:12:03 INFO ExecutorMonitor: New executor 312 has registered 
(new total is 138)
...
21/02/01 15:12:03 INFO TaskSetManager: Starting task 247.0 in stage 4.0 
(TID 2079, 100.100.18.138, executor 312, partition 247, PROCESS_LOCAL, 8777 
bytes)
21/02/01 15:12:03 INFO ExecutorPodsAllocator: Deleting 3 excess pod 
requests (408,312,307).
...
21/02/01 15:12:04 ERROR TaskSchedulerImpl: Lost executor 312 on 
100.100.18.138: The executor with id 312 was deleted by a user or the framework.
21/02/01 15:12:04 INFO TaskSetManager: Task 2079 failed because while it 
was being computed, its executor exited for a reason unrelated to the task. Not 
counting this failure towards the maximum number of failures for the task.
```

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

No.

### How was this patch tested?

 Manually

With this change there was no executor lost with running task on it.

# With unit test

A new test is added and existing test is modified to check these cases.

Closes #31513 from attilapiros/SPARK-34361.

Authored-by: “attilapiros” 
Signed-off-by: Holden Karau 
---
 .../scala/org/apache/spark/deploy/k8s/Config.scala |   3 +-
 .../cluster/k8s/ExecutorPodsAllocator.scala|  66 ++---
 .../k8s/KubernetesClusterSchedulerBackend.scala|   2 +-
 .../cluster/k8s/ExecutorPodsAllocatorSuite.scala   | 155 -
 .../KubernetesClusterSchedulerBackendSuite.scala   |   2 +-
 5 files changed, 204 insertions(+), 24 deletions(-)

diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
index d8fa9e4..8ac2ee4 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
@@ -272,7 +272,8 @@ private[spark] object Config extends Logging {
 
   val KUBERNETES_ALLOCATION_EXECUTOR_TIMEOUT =
 ConfigBuilder("spark.kubernetes.allocation.executor.timeout")
-  .doc("Time to wait before considering a pending executor timedout.")
+  .doc("Time to wait before a newly created executor POD request, which 
does not reached " +
+"the POD pending state yet, considered timedout and will be deleted.")
   .version("3.1.0")
   .timeConf(TimeUnit.MILLISECONDS)
   .checkValue(value => value > 0, "Allocation executor timeout must be a 
positive time value.")
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
index eb35de8..5fc81a6 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
@@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s
 
 import java.time.Instant
 import java.util.concurrent.ConcurrentHash

[spark] branch master updated: [SPARK-32617][K8S][TESTS] Configure kubernetes client based on kubeconfig settings in kubernetes integration tests

2021-02-24 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 b17754a  [SPARK-32617][K8S][TESTS] Configure kubernetes client based 
on kubeconfig settings in kubernetes integration tests
b17754a is described below

commit b17754a8cbd2593eb2b1952e95a7eeb0f8e09cdb
Author: “attilapiros” 
AuthorDate: Wed Feb 24 11:46:27 2021 -0800

[SPARK-32617][K8S][TESTS] Configure kubernetes client based on kubeconfig 
settings in kubernetes integration tests

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

From [minikube version 
v1.1.0](https://github.com/kubernetes/minikube/blob/v1.1.0/CHANGELOG.md) 
kubectl is available as a command. So the kubeconfig settings can be accessed 
like:

```
$ minikube kubectl config view
apiVersion: v1
clusters:
- cluster:
certificate-authority: /Users/attilazsoltpiros/.minikube/ca.crt
server: https://127.0.0.1:32788
  name: minikube
contexts:
- context:
cluster: minikube
namespace: default
user: minikube
  name: minikube
current-context: minikube
kind: Config
preferences: {}
users:
- name: minikube
  user:
client-certificate: 
/Users/attilazsoltpiros/.minikube/profiles/minikube/client.crt
client-key: 
/Users/attilazsoltpiros/.minikube/profiles/minikube/client.key
```

Here the vm-driver was docker and the server port (https://127.0.0.1:32788) 
is different from the hardcoded 8443.

So the main part of this PR is introducing kubernetes client configuration 
based on the kubeconfig (output of `minikube kubectl config view`) in case of 
minikube versions after v1.1.0 and the old legacy way of configuration is also 
kept as minikube version should be supported back to v0.34.1 .

Moreover as the old style of config parsing pattern wasn't sufficient in my 
case as when the `minikube kubectl config view` is called kubectl downloading 
message might be included before the first key I changed it even for the 
existent keys to be a consistent pattern in this file.

The old parsing in an example:
```
private val HOST_PREFIX = "host:"

val hostString = statusString.find(_.contains(s"$HOST_PREFIX "))

val status1 = hostString.get.split(HOST_PREFIX)(1)
```

The new parsing:
```
private val HOST_PREFIX = "host: "

val hostString = statusString.find(_.contains(HOST_PREFIX))

hostString.get.split(HOST_PREFIX)(1)
```

So the PREFIX is extended with the extra space at the declaration (this way 
the two separate string operation are more safe and consistent with each other) 
and the replace is changed to split and getting the 2nd string from the result 
(which is guaranteed to contain only the text after the PREFIX when the PREFIX 
is a contained substring).

Finally there is tiny change in `dev-run-integration-tests.sh` to introduce 
`--skip-building-dependencies` which switchs off building of maven dependencies 
of `kubernetes-integration-tests` from the Spark project.
This could be used when only the `kubernetes-integration-tests` should be 
rebuilded as only the tests are modified.

### Why are the changes needed?

Kubernetes client configuration based on kubeconfig settings is more 
reliable and provides a solution which is minikube version independent.

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

No. This is only test code.

### How was this patch tested?

tested manually on two minikube versions.

Minikube  v0.34.1:

```
$ minikube version
minikube version: v0.34.1

$ grep "version\|building" 
resource-managers/kubernetes/integration-tests/target/integration-tests.log
20/12/12 12:52:25.135 ScalaTest-main-running-DiscoverySuite INFO Minikube: 
minikube version: v0.34.1
20/12/12 12:52:25.761 ScalaTest-main-running-DiscoverySuite INFO Minikube: 
building kubernetes config with apiVersion: v1, masterUrl: 
https://192.168.99.103:8443, caCertFile: 
/Users/attilazsoltpiros/.minikube/ca.crt, clientCertFile: 
/Users/attilazsoltpiros/.minikube/apiserver.crt, clientKeyFile: 
/Users/attilazsoltpiros/.minikube/apiserver.key
```

Minikube v1.15.1
```
$ minikube version

minikube version: v1.15.1
commit: 23f40a012abb52eff365ff99a709501a61ac5876

$ grep "version\|building" 
resource-managers/kubernetes/integration-tests/target/integration-tests.log

20/12/13 06:25:55.086 ScalaTest-main-running-DiscoverySuite INFO Minikube: 
minikube version: v1.15.1
20/12/13 06:25:55.597 ScalaTest-main-running-DiscoverySuite INFO Minikube: 
building kubernetes config with apiVersion: v1,

[spark] branch master updated: [SPARK-33763] Add metrics for better tracking of dynamic allocation

2021-02-17 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 76e5d75  [SPARK-33763] Add metrics for better tracking of dynamic 
allocation
76e5d75 is described below

commit 76e5d75e369609b96248a34bc6015ec61936e652
Author: “attilapiros” 
AuthorDate: Wed Feb 17 13:44:36 2021 -0800

[SPARK-33763] Add metrics for better tracking of dynamic allocation

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

This PR adds the following metrics to track executor remove reasons during 
dynamic allocation:
-  `numberExecutorsGracefullyDecommissioned`: number of executors which 
reached the finished decommissioning state and shut itself down cleanly
- `numberExecutorsDecommissionUnfinished`: executors which requested to 
decommission but they stopped without reaching the finished decommissioning 
state
- `numberExecutorsKilledByDriver`: executors killed by the driver 
(requested to stop)
-  `numberExecutorsExitedUnexpectedly`: executors exited without driver 
request

### Why are the changes needed?

For supporting monitoring of dynamic allocation better with these metrics.

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

Yes. The new metrics will be available for monitoring.

### How was this patch tested?

With unit and integration tests.

Finally manually checked the new metrics in jconsole:
https://user-images.githubusercontent.com/2017933/107458686-de8adf00-6b54-11eb-86f7-41faf2fb638f.png;>

Closes #31450 from attilapiros/SPARK-33763-final.

Authored-by: “attilapiros” 
Signed-off-by: Holden Karau 
---
 .../apache/spark/ExecutorAllocationManager.scala   | 80 +-
 .../executor/CoarseGrainedExecutorBackend.scala|  6 +-
 .../spark/scheduler/ExecutorLossReason.scala   |  4 ++
 .../spark/scheduler/dynalloc/ExecutorMonitor.scala | 20 +-
 .../spark/ExecutorAllocationManagerSuite.scala | 54 ++-
 .../scheduler/dynalloc/ExecutorMonitorSuite.scala  | 22 --
 .../k8s/integrationtest/DecommissionSuite.scala| 11 ++-
 7 files changed, 150 insertions(+), 47 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index bdb768e..822a0a5 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -23,7 +23,7 @@ import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 import scala.util.control.{ControlThrowable, NonFatal}
 
-import com.codahale.metrics.{Gauge, MetricRegistry}
+import com.codahale.metrics.{Counter, Gauge, MetricRegistry}
 
 import org.apache.spark.internal.{config, Logging}
 import org.apache.spark.internal.config._
@@ -135,14 +135,14 @@ private[spark] class ExecutorAllocationManager(
   validateSettings()
 
   // Number of executors to add for each ResourceProfile in the next round
-  private val numExecutorsToAddPerResourceProfileId = new mutable.HashMap[Int, 
Int]
+  private[spark] val numExecutorsToAddPerResourceProfileId = new 
mutable.HashMap[Int, Int]
   numExecutorsToAddPerResourceProfileId(defaultProfileId) = 1
 
   // The desired number of executors at this moment in time. If all our 
executors were to die, this
   // is the number of executors we would immediately want from the cluster 
manager.
   // Note every profile will be allowed to have initial number,
   // we may want to make this configurable per Profile in the future
-  private val numExecutorsTargetPerResourceProfileId = new 
mutable.HashMap[Int, Int]
+  private[spark] val numExecutorsTargetPerResourceProfileId = new 
mutable.HashMap[Int, Int]
   numExecutorsTargetPerResourceProfileId(defaultProfileId) = 
initialNumExecutors
 
   // A timestamp of when an addition should be triggered, or NOT_SET if it is 
not set
@@ -155,14 +155,15 @@ private[spark] class ExecutorAllocationManager(
   // Listener for Spark events that impact the allocation policy
   val listener = new ExecutorAllocationListener
 
-  val executorMonitor = new ExecutorMonitor(conf, client, listenerBus, clock)
-
   // Executor that handles the scheduling task.
   private val executor =
 
ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-dynamic-executor-allocation")
 
   // Metric source for ExecutorAllocationManager to expose internal status to 
MetricsSystem.
-  val executorAllocationManagerSource = new ExecutorAllocationManagerSource
+  val executorAllocationManagerSource = new 
ExecutorAllocationManagerSource(this)
+
+  val executorMonitor =
+new ExecutorMonitor(conf, client, listenerBus, clock, 
executorAllocationManagerSource)
 
   // Whether we are 

[spark] branch master updated: [SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning time & allow decommissioning for excludes

2021-02-09 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 5248ecb  [SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning 
time & allow decommissioning for excludes
5248ecb is described below

commit 5248ecb5ab0c9fdbd5f333e751d1f5fb3c514d14
Author: Holden Karau 
AuthorDate: Tue Feb 9 18:16:09 2021 -0800

[SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning time & allow 
decommissioning for excludes

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

Allow users to have Spark attempt to decommission excluded executors.
Since excluded executors may be flaky, this also adds the ability for users 
to specify a time limit after which a decommissioning executor will be killed 
by Spark.

### Why are the changes needed?

This may help prevent fetch failures from excluded executors, and also 
handle the situation in which executors

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

Yes, two new configuration flags for the behaviour.

### How was this patch tested?

Extended unit and integration tests.

Closes #31539 from holdenk/re=enable-SPARK-34104-SPARK-34105.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../apache/spark/ExecutorAllocationClient.scala|  6 +++
 .../org/apache/spark/internal/config/package.scala | 19 +++-
 .../org/apache/spark/scheduler/HealthTracker.scala | 35 +++---
 .../cluster/CoarseGrainedClusterMessage.scala  |  3 ++
 .../cluster/CoarseGrainedSchedulerBackend.scala| 56 --
 .../spark/scheduler/HealthTrackerSuite.scala   | 45 +
 .../k8s/integrationtest/DecommissionSuite.scala| 32 +
 .../k8s/integrationtest/KubernetesSuite.scala  |  5 +-
 8 files changed, 187 insertions(+), 14 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
index cdba1c4..5b587d7 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -129,6 +129,12 @@ private[spark] trait ExecutorAllocationClient {
 decommissionedExecutors.nonEmpty && 
decommissionedExecutors(0).equals(executorId)
   }
 
+  /**
+   * Request that the cluster manager decommission every executor on the 
specified host.
+   *
+   * @return whether the request is acknowledged by the cluster manager.
+   */
+  def decommissionExecutorsOnHost(host: String): Boolean
 
   /**
* Request that the cluster manager kill every executor on the specified 
host.
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 7aeb51d..3101bb6 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
@@ -827,6 +827,13 @@ package object config {
   .booleanConf
   .createWithDefault(false)
 
+  private[spark] val EXCLUDE_ON_FAILURE_DECOMMISSION_ENABLED =
+ConfigBuilder("spark.excludeOnFailure.killExcludedExecutors.decommission")
+  .doc("Attempt decommission of excluded nodes instead of going directly 
to kill")
+  .version("3.2.0")
+  .booleanConf
+  .createWithDefault(false)
+
   private[spark] val EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF =
 ConfigBuilder("spark.scheduler.executorTaskExcludeOnFailureTime")
   .internal()
@@ -1958,7 +1965,8 @@ package object config {
 
   private[spark] val EXECUTOR_DECOMMISSION_KILL_INTERVAL =
 ConfigBuilder("spark.executor.decommission.killInterval")
-  .doc("Duration after which a decommissioned executor will be killed 
forcefully." +
+  .doc("Duration after which a decommissioned executor will be killed 
forcefully " +
+"*by an outside* (e.g. non-spark) service. " +
 "This config is useful for cloud environments where we know in advance 
when " +
 "an executor is going to go down after decommissioning signal i.e. 
around 2 mins " +
 "in aws spot nodes, 1/2 hrs in spot block nodes etc. This config is 
currently " +
@@ -1967,6 +1975,15 @@ package object config {
   .timeConf(TimeUnit.SECONDS)
   .createOptional
 
+  private[spark] val EXECUTOR_DECOMMISSION_FORCE_KILL_TIMEOUT =
+ConfigBuilder("spark.executor.decommission.forceKillTimeout")
+  .doc("Duration after which a Spark will force a decommissioning executor 
to exit." +
+" this should be set to a high value in most 

[spark] branch master updated: [SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning time & allow decommissioning for excludes

2021-02-09 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 50641d2  [SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning 
time & allow decommissioning for excludes
50641d2 is described below

commit 50641d2e3d659f51432aa2c0e6b9af76d71a5796
Author: Holden Karau 
AuthorDate: Tue Feb 9 14:21:24 2021 -0800

[SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning time & allow 
decommissioning for excludes

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

Allow users to have Spark attempt to decommission excluded executors.
Since excluded executors may be flaky, this also adds the ability for users 
to specify a time limit after which a decommissioning executor will be killed 
by Spark.

### Why are the changes needed?

This may help prevent fetch failures from excluded executors, and also 
handle the situation in which executors

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

Yes, two new configuration flags for the behaviour.

### How was this patch tested?

Extended unit and integration tests.

Closes #31249 from 
holdenk/configure-inaccessibleList-kill-to-use-decommissioning.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../apache/spark/ExecutorAllocationClient.scala|  6 +++
 .../org/apache/spark/internal/config/package.scala | 19 +++-
 .../org/apache/spark/scheduler/HealthTracker.scala | 35 +++---
 .../cluster/CoarseGrainedClusterMessage.scala  |  3 ++
 .../cluster/CoarseGrainedSchedulerBackend.scala| 56 --
 .../spark/scheduler/HealthTrackerSuite.scala   | 45 +
 .../k8s/integrationtest/DecommissionSuite.scala| 32 +
 .../k8s/integrationtest/KubernetesSuite.scala  |  5 +-
 8 files changed, 187 insertions(+), 14 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
index cdba1c4..5b587d7 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -129,6 +129,12 @@ private[spark] trait ExecutorAllocationClient {
 decommissionedExecutors.nonEmpty && 
decommissionedExecutors(0).equals(executorId)
   }
 
+  /**
+   * Request that the cluster manager decommission every executor on the 
specified host.
+   *
+   * @return whether the request is acknowledged by the cluster manager.
+   */
+  def decommissionExecutorsOnHost(host: String): Boolean
 
   /**
* Request that the cluster manager kill every executor on the specified 
host.
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 7aeb51d..3101bb6 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
@@ -827,6 +827,13 @@ package object config {
   .booleanConf
   .createWithDefault(false)
 
+  private[spark] val EXCLUDE_ON_FAILURE_DECOMMISSION_ENABLED =
+ConfigBuilder("spark.excludeOnFailure.killExcludedExecutors.decommission")
+  .doc("Attempt decommission of excluded nodes instead of going directly 
to kill")
+  .version("3.2.0")
+  .booleanConf
+  .createWithDefault(false)
+
   private[spark] val EXCLUDE_ON_FAILURE_LEGACY_TIMEOUT_CONF =
 ConfigBuilder("spark.scheduler.executorTaskExcludeOnFailureTime")
   .internal()
@@ -1958,7 +1965,8 @@ package object config {
 
   private[spark] val EXECUTOR_DECOMMISSION_KILL_INTERVAL =
 ConfigBuilder("spark.executor.decommission.killInterval")
-  .doc("Duration after which a decommissioned executor will be killed 
forcefully." +
+  .doc("Duration after which a decommissioned executor will be killed 
forcefully " +
+"*by an outside* (e.g. non-spark) service. " +
 "This config is useful for cloud environments where we know in advance 
when " +
 "an executor is going to go down after decommissioning signal i.e. 
around 2 mins " +
 "in aws spot nodes, 1/2 hrs in spot block nodes etc. This config is 
currently " +
@@ -1967,6 +1975,15 @@ package object config {
   .timeConf(TimeUnit.SECONDS)
   .createOptional
 
+  private[spark] val EXECUTOR_DECOMMISSION_FORCE_KILL_TIMEOUT =
+ConfigBuilder("spark.executor.decommission.forceKillTimeout")
+  .doc("Duration after which a Spark will force a decommissioning executor 
to exit." +
+ 

[spark] branch master updated: [SPARK-34363][CORE] Add an option for limiting storage for migrated shuffle blocks

2021-02-09 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 2b51843  [SPARK-34363][CORE] Add an option for limiting storage for 
migrated shuffle blocks
2b51843 is described below

commit 2b51843ca41236f8cec29c406ea35ce1088364cf
Author: Holden Karau 
AuthorDate: Tue Feb 9 10:21:56 2021 -0800

[SPARK-34363][CORE] Add an option for limiting storage for migrated shuffle 
blocks

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

Allow users to configure a maximum amount of shuffle blocks to be stored 
and reject remote shuffle blocks when this threshold is exceeded.

### Why are the changes needed?

In disk constrained environments with large amount of shuffle data, 
migrations may result in excessive disk pressure on the nodes. On Kube nodes 
this can result in cascading failures when combined with `emptyDir`.

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

Yes, new configuration parameter.

### How was this patch tested?

New unit tests.

Closes #31493 from 
holdenk/SPARK-34337-reject-disk-blocks-when-under-disk-pressure.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../org/apache/spark/internal/config/package.scala | 11 ++
 .../spark/shuffle/IndexShuffleBlockResolver.scala  | 20 +-
 .../apache/spark/shuffle/MigratableResolver.scala  |  1 +
 .../apache/spark/storage/BlockManagerSuite.scala   | 46 --
 4 files changed, 65 insertions(+), 13 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 1afad30..7aeb51d 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
@@ -488,6 +488,17 @@ package object config {
   .booleanConf
   .createWithDefault(false)
 
+  private[spark] val STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE =
+ConfigBuilder("spark.storage.decommission.shuffleBlocks.maxDiskSize")
+  .doc("Maximum disk space to use to store shuffle blocks before rejecting 
remote " +
+"shuffle blocks. Rejecting remote shuffle blocks means that an 
executor will not receive " +
+"any shuffle migrations, and if there are no other executors available 
for migration " +
+"then shuffle blocks will be lost unless " +
+s"${STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH.key} is configured.")
+  .version("3.2.0")
+  .bytesConf(ByteUnit.BYTE)
+  .createOptional
+
   private[spark] val STORAGE_REPLICATION_TOPOLOGY_FILE =
 ConfigBuilder("spark.storage.replication.topologyFile")
   .version("2.1.0")
diff --git 
a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala 
b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
index 5f0bb42..d30b73a 100644
--- 
a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
+++ 
b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
@@ -22,8 +22,8 @@ import java.nio.ByteBuffer
 import java.nio.channels.Channels
 import java.nio.file.Files
 
-import org.apache.spark.{SparkConf, SparkEnv}
-import org.apache.spark.internal.Logging
+import org.apache.spark.{SparkConf, SparkEnv, SparkException}
+import org.apache.spark.internal.{config, Logging}
 import org.apache.spark.io.NioBufferedFileInputStream
 import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, 
ManagedBuffer}
 import org.apache.spark.network.client.StreamCallbackWithID
@@ -56,6 +56,8 @@ private[spark] class IndexShuffleBlockResolver(
 
   private val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle")
 
+  private val remoteShuffleMaxDisk: Option[Long] =
+conf.get(config.STORAGE_DECOMMISSION_SHUFFLE_MAX_DISK_SIZE)
 
   def getDataFile(shuffleId: Int, mapId: Long): File = getDataFile(shuffleId, 
mapId, None)
 
@@ -72,6 +74,13 @@ private[spark] class IndexShuffleBlockResolver(
 }
   }
 
+  private def getShuffleBytesStored(): Long = {
+val shuffleFiles: Seq[File] = getStoredShuffles().map {
+  si => getDataFile(si.shuffleId, si.mapId)
+}
+shuffleFiles.map(_.length()).sum
+  }
+
   /**
* Get the shuffle data file.
*
@@ -173,6 +182,13 @@ private[spark] class IndexShuffleBlockResolver(
*/
   override def putShuffleBlockAsStream(blockId: BlockId, serializerManager: 
SerializerManager):
   StreamCallbackWithID = {
+// Throw an exception if we have exceeded maximum shuffle files stored
+remoteShuffleMaxDisk.foreach { maxBytes 

[spark] branch master updated: [SPARK-34209][SQL] Delegate table name validation to the session catalog

2021-02-09 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 cf7a13c  [SPARK-34209][SQL] Delegate table name validation to the 
session catalog
cf7a13c is described below

commit cf7a13c363ef5d56556c9d70e7811bf6a40de55f
Author: Holden Karau 
AuthorDate: Tue Feb 9 10:15:16 2021 -0800

[SPARK-34209][SQL] Delegate table name validation to the session catalog

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

Delegate table name validation to the session catalog

### Why are the changes needed?

Queerying of tables with nested namespaces.

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

SQL queries of nested namespace queries

### How was this patch tested?

Unit tests updated.

Closes #31427 from 
holdenk/SPARK-34209-delegate-table-name-validation-to-the-catalog.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../sql/connector/catalog/LookupCatalog.scala  |  5 --
 .../spark/sql/connector/DataSourceV2SQLSuite.scala | 58 ++
 .../spark/sql/execution/command/DDLSuite.scala |  2 +-
 3 files changed, 38 insertions(+), 27 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
index 16416fa..af951a0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/LookupCatalog.scala
@@ -57,14 +57,9 @@ private[sql] trait LookupCatalog extends Logging {
* Extract session catalog and identifier from a multi-part identifier.
*/
   object SessionCatalogAndIdentifier {
-import 
org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
 
 def unapply(parts: Seq[String]): Option[(CatalogPlugin, Identifier)] = 
parts match {
   case CatalogAndIdentifier(catalog, ident) if 
CatalogV2Util.isSessionCatalog(catalog) =>
-if (ident.namespace.length != 1) {
-  throw new AnalysisException(
-s"The namespace in session catalog must have exactly one name 
part: ${parts.quoted}")
-}
 Some(catalog, ident)
   case _ => None
 }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index 4d41020..7d67e1c 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -2085,8 +2085,7 @@ class DataSourceV2SQLSuite
 val e1 = intercept[AnalysisException] {
   sql("DESCRIBE FUNCTION default.ns1.ns2.fun")
 }
-assert(e1.message.contains(
-  "The namespace in session catalog must have exactly one name part: 
default.ns1.ns2.fun"))
+assert(e1.message.contains("Unsupported function name 
'default.ns1.ns2.fun'"))
   }
 
   test("SHOW FUNCTIONS not valid v1 namespace") {
@@ -2107,8 +2106,7 @@ class DataSourceV2SQLSuite
 val e1 = intercept[AnalysisException] {
   sql("DROP FUNCTION default.ns1.ns2.fun")
 }
-assert(e1.message.contains(
-  "The namespace in session catalog must have exactly one name part: 
default.ns1.ns2.fun"))
+assert(e1.message.contains("Unsupported function name 
'default.ns1.ns2.fun'"))
   }
 
   test("CREATE FUNCTION: only support session catalog") {
@@ -2120,8 +2118,7 @@ class DataSourceV2SQLSuite
 val e1 = intercept[AnalysisException] {
   sql("CREATE FUNCTION default.ns1.ns2.fun as 'f'")
 }
-assert(e1.message.contains(
-  "The namespace in session catalog must have exactly one name part: 
default.ns1.ns2.fun"))
+assert(e1.message.contains("Unsupported function name 
'default.ns1.ns2.fun'"))
   }
 
   test("REFRESH FUNCTION: only support session catalog") {
@@ -2134,7 +2131,7 @@ class DataSourceV2SQLSuite
   sql("REFRESH FUNCTION default.ns1.ns2.fun")
 }
 assert(e1.message.contains(
-  "The namespace in session catalog must have exactly one name part: 
default.ns1.ns2.fun"))
+  "Unsupported function name 'default.ns1.ns2.fun'"))
   }
 
   test("global temp view should not be masked by v2 catalog") {
@@ -2172,7 +2169,7 @@ class DataSourceV2SQLSuite
   sql(s"CREATE TABLE $globalTempDB.ns1.ns2.tbl (id bigint, data string) 
USING json")
 }
 assert(e.message.contains(
-  "The namespace in session catalog must have exa

[spark] branch branch-3.1 updated: [SPARK-34334][K8S] Correctly identify timed out pending pod requests as excess request

2021-02-09 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 5d0c84a  [SPARK-34334][K8S] Correctly identify timed out pending pod 
requests as excess request
5d0c84a is described below

commit 5d0c84adaf10e2e946b988b51a95cd8a11e795c6
Author: “attilapiros” 
AuthorDate: Tue Feb 9 10:06:55 2021 -0800

[SPARK-34334][K8S] Correctly identify timed out pending pod requests as 
excess request

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

Fixing identification of timed-out pending pod requests as excess requests 
to delete when the excess is higher than the newly created timed out requests 
and there is some non-timed out newly created requests too.

### Why are the changes needed?

After https://github.com/apache/spark/pull/29981 only timed out newly 
created requests and timed out pending requests are taken as excess request.

But there is small bug when the excess is higher than the newly created 
timed out requests and there is some non-timed out newly created requests as 
well. Because all the newly created requests are counted as excess request when 
items are chosen from the timed out pod pending requests.

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

No.

### How was this patch tested?

 There is new unit test added: `SPARK-34334: correctly identify timed out 
pending pod requests as excess`.

Closes #31445 from attilapiros/SPARK-34334.

Authored-by: “attilapiros” 
Signed-off-by: Holden Karau 
(cherry picked from commit b2dc38b6546552cf3fcfdcd466d7d04d9aa3078c)
Signed-off-by: Holden Karau 
---
 .../cluster/k8s/ExecutorPodsAllocator.scala| 11 +++---
 .../cluster/k8s/ExecutorPodsAllocatorSuite.scala   | 45 ++
 2 files changed, 51 insertions(+), 5 deletions(-)

diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
index f4cd2d0..eb35de8 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
@@ -223,14 +223,15 @@ private[spark] class ExecutorPodsAllocator(
 
   if (knownPodCount > targetNum) {
 val excess = knownPodCount - targetNum
+val newlyCreatedToDelete = newlyCreatedExecutorsForRpId
+  .filter { case (_, (_, createTime)) =>
+currentTime - createTime > executorIdleTimeout
+  }.keys.take(excess).toList
 val knownPendingToDelete = currentPendingExecutors
   .filter(x => isExecutorIdleTimedOut(x._2, currentTime))
+  .take(excess - newlyCreatedToDelete.size)
   .map { case (id, _) => id }
-  .take(excess - newlyCreatedExecutorsForRpId.size)
-val toDelete = newlyCreatedExecutorsForRpId
-  .filter { case (_, (_, createTime)) =>
-currentTime - createTime > executorIdleTimeout
-  }.keys.take(excess).toList ++ knownPendingToDelete
+val toDelete = newlyCreatedToDelete ++ knownPendingToDelete
 
 if (toDelete.nonEmpty) {
   logInfo(s"Deleting ${toDelete.size} excess pod requests 
(${toDelete.mkString(",")}).")
diff --git 
a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
 
b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
index 8401f71..c7e2f1a 100644
--- 
a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
+++ 
b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
@@ -216,6 +216,51 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite 
with BeforeAndAfter {
 assert(!podsAllocatorUnderTest.isDeleted("4"))
   }
 
+  test("SPARK-34334: correctly identify timed out pending pod requests as 
excess") {
+when(podOperations
+  .withField("status.phase", "Pending"))
+  .thenReturn(podOperations)
+when(podOperations
+  .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID))
+  .thenReturn(podOperations)
+when(podOperations
+  .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE))
+  .thenReturn(podOperations)
+when(podOperations
+  .withLabelIn(meq(SPARK_EXECUTOR_ID_LABEL), any()))
+  .thenReturn(podOperations)
+
+val sta

[spark] branch master updated: [SPARK-34334][K8S] Correctly identify timed out pending pod requests as excess request

2021-02-09 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 b2dc38b  [SPARK-34334][K8S] Correctly identify timed out pending pod 
requests as excess request
b2dc38b is described below

commit b2dc38b6546552cf3fcfdcd466d7d04d9aa3078c
Author: “attilapiros” 
AuthorDate: Tue Feb 9 10:06:55 2021 -0800

[SPARK-34334][K8S] Correctly identify timed out pending pod requests as 
excess request

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

Fixing identification of timed-out pending pod requests as excess requests 
to delete when the excess is higher than the newly created timed out requests 
and there is some non-timed out newly created requests too.

### Why are the changes needed?

After https://github.com/apache/spark/pull/29981 only timed out newly 
created requests and timed out pending requests are taken as excess request.

But there is small bug when the excess is higher than the newly created 
timed out requests and there is some non-timed out newly created requests as 
well. Because all the newly created requests are counted as excess request when 
items are chosen from the timed out pod pending requests.

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

No.

### How was this patch tested?

 There is new unit test added: `SPARK-34334: correctly identify timed out 
pending pod requests as excess`.

Closes #31445 from attilapiros/SPARK-34334.

Authored-by: “attilapiros” 
Signed-off-by: Holden Karau 
---
 .../cluster/k8s/ExecutorPodsAllocator.scala| 11 +++---
 .../cluster/k8s/ExecutorPodsAllocatorSuite.scala   | 45 ++
 2 files changed, 51 insertions(+), 5 deletions(-)

diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
index f4cd2d0..eb35de8 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala
@@ -223,14 +223,15 @@ private[spark] class ExecutorPodsAllocator(
 
   if (knownPodCount > targetNum) {
 val excess = knownPodCount - targetNum
+val newlyCreatedToDelete = newlyCreatedExecutorsForRpId
+  .filter { case (_, (_, createTime)) =>
+currentTime - createTime > executorIdleTimeout
+  }.keys.take(excess).toList
 val knownPendingToDelete = currentPendingExecutors
   .filter(x => isExecutorIdleTimedOut(x._2, currentTime))
+  .take(excess - newlyCreatedToDelete.size)
   .map { case (id, _) => id }
-  .take(excess - newlyCreatedExecutorsForRpId.size)
-val toDelete = newlyCreatedExecutorsForRpId
-  .filter { case (_, (_, createTime)) =>
-currentTime - createTime > executorIdleTimeout
-  }.keys.take(excess).toList ++ knownPendingToDelete
+val toDelete = newlyCreatedToDelete ++ knownPendingToDelete
 
 if (toDelete.nonEmpty) {
   logInfo(s"Deleting ${toDelete.size} excess pod requests 
(${toDelete.mkString(",")}).")
diff --git 
a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
 
b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
index d4d8980..eaf5fd8 100644
--- 
a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
+++ 
b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala
@@ -216,6 +216,51 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite 
with BeforeAndAfter {
 assert(!podsAllocatorUnderTest.isDeleted("4"))
   }
 
+  test("SPARK-34334: correctly identify timed out pending pod requests as 
excess") {
+when(podOperations
+  .withField("status.phase", "Pending"))
+  .thenReturn(podOperations)
+when(podOperations
+  .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID))
+  .thenReturn(podOperations)
+when(podOperations
+  .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE))
+  .thenReturn(podOperations)
+when(podOperations
+  .withLabelIn(meq(SPARK_EXECUTOR_ID_LABEL), any()))
+  .thenReturn(podOperations)
+
+val startTime = Instant.now.toEpochMilli
+waitForExecutorPodsClock.setTime(startTime)
+
+podsAllocatorUnderTest.setT

[spark] branch branch-3.1 updated: [SPARK-32866][K8S] Fix docker cross-build

2021-01-28 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 697bdca  [SPARK-32866][K8S] Fix docker cross-build
697bdca is described below

commit 697bdca292105a9e6efaa1f44e8aec3478bbb3ae
Author: Holden Karau 
AuthorDate: Thu Jan 28 11:57:42 2021 -0800

[SPARK-32866][K8S] Fix docker cross-build

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

Add `--push` to the docker script for buildx

### Why are the changes needed?

Doing a separate docker push with `buildx` images doesn't work.

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

Automatically pushes work when cross-building.

### How was this patch tested?

cross-built docker containers

Closes #31299 from holdenk/SPARK-32866-docker-buildx-update.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
(cherry picked from commit 497f599a37d2250c14a7bd2699bd3ac65bd08a58)
Signed-off-by: Holden Karau 
---
 bin/docker-image-tool.sh | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh
index 2ec1ab8..83b13b8 100755
--- a/bin/docker-image-tool.sh
+++ b/bin/docker-image-tool.sh
@@ -181,7 +181,7 @@ function build {
 error "Failed to build Spark JVM Docker image, please refer to Docker 
build output for details."
   fi
   if [ "${CROSS_BUILD}" != "false" ]; then
-  (cd $(img_ctx_dir base) && docker buildx build $ARCHS $NOCACHEARG 
"${BUILD_ARGS[@]}" \
+  (cd $(img_ctx_dir base) && docker buildx build $ARCHS $NOCACHEARG 
"${BUILD_ARGS[@]}" --push \
 -t $(image_ref spark) \
 -f "$BASEDOCKERFILE" .)
   fi
@@ -194,7 +194,7 @@ function build {
 error "Failed to build PySpark Docker image, please refer to Docker 
build output for details."
   fi
   if [ "${CROSS_BUILD}" != "false" ]; then
-(cd $(img_ctx_dir pyspark) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
+(cd $(img_ctx_dir pyspark) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" --push \
   -t $(image_ref spark-py) \
   -f "$PYDOCKERFILE" .)
   fi
@@ -208,7 +208,7 @@ function build {
   error "Failed to build SparkR Docker image, please refer to Docker build 
output for details."
 fi
 if [ "${CROSS_BUILD}" != "false" ]; then
-  (cd $(img_ctx_dir sparkr) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
+  (cd $(img_ctx_dir sparkr) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" --push \
 -t $(image_ref spark-r) \
 -f "$RDOCKERFILE" .)
 fi


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



[spark] branch master updated: [SPARK-32866][K8S] Fix docker cross-build

2021-01-28 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 497f599  [SPARK-32866][K8S] Fix docker cross-build
497f599 is described below

commit 497f599a37d2250c14a7bd2699bd3ac65bd08a58
Author: Holden Karau 
AuthorDate: Thu Jan 28 11:57:42 2021 -0800

[SPARK-32866][K8S] Fix docker cross-build

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

Add `--push` to the docker script for buildx

### Why are the changes needed?

Doing a separate docker push with `buildx` images doesn't work.

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

Automatically pushes work when cross-building.

### How was this patch tested?

cross-built docker containers

Closes #31299 from holdenk/SPARK-32866-docker-buildx-update.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 bin/docker-image-tool.sh | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh
index 2ec1ab8..83b13b8 100755
--- a/bin/docker-image-tool.sh
+++ b/bin/docker-image-tool.sh
@@ -181,7 +181,7 @@ function build {
 error "Failed to build Spark JVM Docker image, please refer to Docker 
build output for details."
   fi
   if [ "${CROSS_BUILD}" != "false" ]; then
-  (cd $(img_ctx_dir base) && docker buildx build $ARCHS $NOCACHEARG 
"${BUILD_ARGS[@]}" \
+  (cd $(img_ctx_dir base) && docker buildx build $ARCHS $NOCACHEARG 
"${BUILD_ARGS[@]}" --push \
 -t $(image_ref spark) \
 -f "$BASEDOCKERFILE" .)
   fi
@@ -194,7 +194,7 @@ function build {
 error "Failed to build PySpark Docker image, please refer to Docker 
build output for details."
   fi
   if [ "${CROSS_BUILD}" != "false" ]; then
-(cd $(img_ctx_dir pyspark) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
+(cd $(img_ctx_dir pyspark) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" --push \
   -t $(image_ref spark-py) \
   -f "$PYDOCKERFILE" .)
   fi
@@ -208,7 +208,7 @@ function build {
   error "Failed to build SparkR Docker image, please refer to Docker build 
output for details."
 fi
 if [ "${CROSS_BUILD}" != "false" ]; then
-  (cd $(img_ctx_dir sparkr) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
+  (cd $(img_ctx_dir sparkr) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" --push \
 -t $(image_ref spark-r) \
 -f "$RDOCKERFILE" .)
 fi


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



[spark] branch branch-3.1 updated: [SPARK-33711][K8S] Avoid race condition between POD lifecycle manager and scheduler backend

2021-01-11 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 7da0164  [SPARK-33711][K8S] Avoid race condition between POD lifecycle 
manager and scheduler backend
7da0164 is described below

commit 7da0164824dc6d99d48dcddfcbaef987fe09765e
Author: “attilapiros” 
AuthorDate: Mon Jan 11 14:25:12 2021 -0800

[SPARK-33711][K8S] Avoid race condition between POD lifecycle manager and 
scheduler backend

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

Missing POD detection is extended by timestamp (and time limit) based check 
to avoid wrongfully detection of missing POD detection.

The two new timestamps:
- `fullSnapshotTs` is introduced for the `ExecutorPodsSnapshot` which only 
updated by the pod polling snapshot source
- `registrationTs` is introduced for the `ExecutorData` and it is 
initialized at the executor registration at the scheduler backend

Moreover a new config `spark.kubernetes.executor.missingPodDetectDelta` is 
used to specify the accepted delta between the two.

### Why are the changes needed?

Watching a POD (`ExecutorPodsWatchSnapshotSource`) only inform about single 
POD changes. This could wrongfully lead to detecting of missing PODs (PODs 
known by scheduler backend but missing from POD snapshots) by the executor POD 
lifecycle manager.

A key indicator of this error is seeing this log message:

> "The executor with ID [some_id] was not found in the cluster but we 
didn't get a reason why. Marking the executor as failed. The executor may have 
been deleted but the driver missed the deletion event."

So one of the problem is running the missing POD detection check even when 
a single POD is changed without having a full consistent snapshot about all the 
PODs (see `ExecutorPodsPollingSnapshotSource`).
The other problem could be the race between the executor POD lifecycle 
manager and the scheduler backend: so even in case of a having a full snapshot 
the registration at the scheduler backend could precede the snapshot polling 
(and processing of those polled snapshots).

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

Yes. When the POD is missing then the reason message explaining the 
executor's exit is extended with both timestamps (the polling time and the 
executor registration time) and even the new config is mentioned.

### How was this patch tested?

The existing unit tests are extended.

Closes #30675 from attilapiros/SPARK-33711.

Authored-by: “attilapiros” 
Signed-off-by: Holden Karau 
(cherry picked from commit 6bd7a6200f8beaab1c68b2469df05870ea788d49)
Signed-off-by: Holden Karau 
---
 .../cluster/CoarseGrainedSchedulerBackend.scala|  6 ++-
 .../spark/scheduler/cluster/ExecutorData.scala |  4 +-
 .../scala/org/apache/spark/deploy/k8s/Config.scala | 11 +
 .../cluster/k8s/ExecutorPodsAllocator.scala|  2 +-
 .../cluster/k8s/ExecutorPodsLifecycleManager.scala | 50 +++---
 .../cluster/k8s/ExecutorPodsSnapshot.scala | 12 +++---
 .../k8s/ExecutorPodsSnapshotsStoreImpl.scala   |  8 +++-
 .../DeterministicExecutorPodsSnapshotsStore.scala  |  8 ++--
 .../k8s/ExecutorPodsLifecycleManagerSuite.scala| 22 +++---
 .../cluster/k8s/ExecutorPodsSnapshotSuite.scala|  4 +-
 .../k8s/ExecutorPodsSnapshotsStoreSuite.scala  | 33 --
 11 files changed, 101 insertions(+), 59 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 2bd0b4c..ccb5eb1 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -239,7 +239,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   }
   val data = new ExecutorData(executorRef, executorAddress, hostname,
 0, cores, logUrlHandler.applyPattern(logUrls, attributes), 
attributes,
-resourcesInfo, resourceProfileId)
+resourcesInfo, resourceProfileId, registrationTs = 
System.currentTimeMillis())
   // This must be synchronized because variables mutated
   // in this block are read when requesting executors
   CoarseGrainedSchedulerBackend.this.synchronized {
@@ -629,6 +629,10 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 executorDataMap.keySet.toSeq
   }
 
+  def getExecutorsWithRegistrationTs(): Map[String, Long] = synchronized {
+executorDataMap.mapValues(v => v.regis

[spark] branch master updated: [SPARK-33711][K8S] Avoid race condition between POD lifecycle manager and scheduler backend

2021-01-11 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 6bd7a62  [SPARK-33711][K8S] Avoid race condition between POD lifecycle 
manager and scheduler backend
6bd7a62 is described below

commit 6bd7a6200f8beaab1c68b2469df05870ea788d49
Author: “attilapiros” 
AuthorDate: Mon Jan 11 14:25:12 2021 -0800

[SPARK-33711][K8S] Avoid race condition between POD lifecycle manager and 
scheduler backend

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

Missing POD detection is extended by timestamp (and time limit) based check 
to avoid wrongfully detection of missing POD detection.

The two new timestamps:
- `fullSnapshotTs` is introduced for the `ExecutorPodsSnapshot` which only 
updated by the pod polling snapshot source
- `registrationTs` is introduced for the `ExecutorData` and it is 
initialized at the executor registration at the scheduler backend

Moreover a new config `spark.kubernetes.executor.missingPodDetectDelta` is 
used to specify the accepted delta between the two.

### Why are the changes needed?

Watching a POD (`ExecutorPodsWatchSnapshotSource`) only inform about single 
POD changes. This could wrongfully lead to detecting of missing PODs (PODs 
known by scheduler backend but missing from POD snapshots) by the executor POD 
lifecycle manager.

A key indicator of this error is seeing this log message:

> "The executor with ID [some_id] was not found in the cluster but we 
didn't get a reason why. Marking the executor as failed. The executor may have 
been deleted but the driver missed the deletion event."

So one of the problem is running the missing POD detection check even when 
a single POD is changed without having a full consistent snapshot about all the 
PODs (see `ExecutorPodsPollingSnapshotSource`).
The other problem could be the race between the executor POD lifecycle 
manager and the scheduler backend: so even in case of a having a full snapshot 
the registration at the scheduler backend could precede the snapshot polling 
(and processing of those polled snapshots).

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

Yes. When the POD is missing then the reason message explaining the 
executor's exit is extended with both timestamps (the polling time and the 
executor registration time) and even the new config is mentioned.

### How was this patch tested?

The existing unit tests are extended.

Closes #30675 from attilapiros/SPARK-33711.

Authored-by: “attilapiros” 
Signed-off-by: Holden Karau 
---
 .../cluster/CoarseGrainedSchedulerBackend.scala|  6 ++-
 .../spark/scheduler/cluster/ExecutorData.scala |  4 +-
 .../scala/org/apache/spark/deploy/k8s/Config.scala | 11 +
 .../cluster/k8s/ExecutorPodsAllocator.scala|  2 +-
 .../cluster/k8s/ExecutorPodsLifecycleManager.scala | 50 +++---
 .../cluster/k8s/ExecutorPodsSnapshot.scala | 12 +++---
 .../k8s/ExecutorPodsSnapshotsStoreImpl.scala   |  8 +++-
 .../DeterministicExecutorPodsSnapshotsStore.scala  |  8 ++--
 .../k8s/ExecutorPodsLifecycleManagerSuite.scala| 22 +++---
 .../cluster/k8s/ExecutorPodsSnapshotSuite.scala|  4 +-
 .../k8s/ExecutorPodsSnapshotsStoreSuite.scala  | 33 --
 11 files changed, 101 insertions(+), 59 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 2bd0b4c..ccb5eb1 100644
--- 
a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -239,7 +239,7 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
   }
   val data = new ExecutorData(executorRef, executorAddress, hostname,
 0, cores, logUrlHandler.applyPattern(logUrls, attributes), 
attributes,
-resourcesInfo, resourceProfileId)
+resourcesInfo, resourceProfileId, registrationTs = 
System.currentTimeMillis())
   // This must be synchronized because variables mutated
   // in this block are read when requesting executors
   CoarseGrainedSchedulerBackend.this.synchronized {
@@ -629,6 +629,10 @@ class CoarseGrainedSchedulerBackend(scheduler: 
TaskSchedulerImpl, val rpcEnv: Rp
 executorDataMap.keySet.toSeq
   }
 
+  def getExecutorsWithRegistrationTs(): Map[String, Long] = synchronized {
+executorDataMap.mapValues(v => v.registrationTs).toMap
+  }
+
   override def isExecutorActive(id: String): Boolean = synchronized {
 executorDat

[spark] branch master updated: [SPARK-33261][K8S] Add a developer API for custom feature steps

2020-12-14 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 5885cc1  [SPARK-33261][K8S] Add a developer API for custom feature 
steps
5885cc1 is described below

commit 5885cc15cae9c9780530e235d2bd4bd6beda5dbb
Author: Holden Karau 
AuthorDate: Mon Dec 14 12:05:28 2020 -0800

[SPARK-33261][K8S] Add a developer API for custom feature steps

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

Add a developer API for custom driver & executor feature steps.

### Why are the changes needed?

While we allow templates for the basis of pod creation, some deployments 
need more flexibility in how the pods are configured. This adds a developer API 
for custom deployments.

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

New developer API.

### How was this patch tested?

Extended tests to verify custom step is applied when configured.

Closes #30206 from 
holdenk/SPARK-33261-allow-people-to-extend-pod-feature-steps.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../scala/org/apache/spark/deploy/k8s/Config.scala | 20 ++
 .../org/apache/spark/deploy/k8s/SparkPod.scala | 11 +++-
 .../k8s/features/KubernetesFeatureConfigStep.scala |  7 +-
 .../k8s/submit/KubernetesDriverBuilder.scala   |  8 ++-
 .../cluster/k8s/KubernetesExecutorBuilder.scala|  8 ++-
 .../apache/spark/deploy/k8s/PodBuilderSuite.scala  | 76 ++
 .../k8s/submit/KubernetesDriverBuilderSuite.scala  |  5 +-
 .../k8s/KubernetesExecutorBuilderSuite.scala   |  4 ++
 8 files changed, 134 insertions(+), 5 deletions(-)

diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
index c28d6fd..40609ae 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
@@ -219,6 +219,26 @@ private[spark] object Config extends Logging {
   .stringConf
   .createOptional
 
+  val KUBERNETES_DRIVER_POD_FEATURE_STEPS =
+ConfigBuilder("spark.kubernetes.driver.pod.featureSteps")
+  .doc("Class names of an extra driver pod feature step implementing " +
+"KubernetesFeatureConfigStep. This is a developer API. Comma 
separated. " +
+"Runs after all of Spark internal feature steps.")
+  .version("3.2.0")
+  .stringConf
+  .toSequence
+  .createWithDefault(Nil)
+
+  val KUBERNETES_EXECUTOR_POD_FEATURE_STEPS =
+ConfigBuilder("spark.kubernetes.executor.pod.featureSteps")
+  .doc("Class name of an extra executor pod feature step implementing " +
+"KubernetesFeatureConfigStep. This is a developer API. Comma 
separated. " +
+"Runs after all of Spark internal feature steps.")
+  .version("3.2.0")
+  .stringConf
+  .toSequence
+  .createWithDefault(Nil)
+
   val KUBERNETES_ALLOCATION_BATCH_SIZE =
 ConfigBuilder("spark.kubernetes.allocation.batch.size")
   .doc("Number of pods to launch at once in each round of executor 
allocation.")
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala
index fd11963..c2298e7 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala
@@ -18,7 +18,16 @@ package org.apache.spark.deploy.k8s
 
 import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, 
PodBuilder}
 
-private[spark] case class SparkPod(pod: Pod, container: Container) {
+import org.apache.spark.annotation.{DeveloperApi, Unstable}
+
+/**
+ * :: DeveloperApi ::
+ *
+ * Represents a SparkPod consisting of pod and the container within the pod.
+ */
+@Unstable
+@DeveloperApi
+case class SparkPod(pod: Pod, container: Container) {
 
   /**
* Convenience method to apply a series of chained transformations to a pod.
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KubernetesFeatureConfigStep.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KubernetesFeatureConfigStep.scala
index 58cdaa3..3fec926 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KubernetesFeatureConfigStep.scala
+++ 
b/resource-managers/kuber

[spark] branch master updated: [SPARK-33716][K8S] Fix potential race condition during pod termination

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

holden 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 15486fa  [SPARK-33716][K8S] Fix potential race condition during pod 
termination
15486fa is described below

commit 15486fa970aa104e285cae0379a110f3795f3eaa
Author: Holden Karau 
AuthorDate: Fri Dec 11 14:43:57 2020 -0800

[SPARK-33716][K8S] Fix potential race condition during pod termination

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

Check that the pod state is not pending or running even if there is a 
deletion timestamp.

### Why are the changes needed?

This can occur when the pod state and deletion timestamp are not updated by 
etcd in sync & we get a pod snapshot during an inconsistent view.

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

No

### How was this patch tested?

Manual testing with local version of Minikube on an overloaded computer 
that caused out of sync updates.

Closes #30693 from 
holdenk/SPARK-33716-decommissioning-race-condition-during-pod-snapshot.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala
index be75311..e81d213 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala
@@ -93,7 +93,8 @@ object ExecutorPodsSnapshot extends Logging {
   (
 pod.getStatus == null ||
 pod.getStatus.getPhase == null ||
-pod.getStatus.getPhase.toLowerCase(Locale.ROOT) != "terminating"
+  (pod.getStatus.getPhase.toLowerCase(Locale.ROOT) != "terminating" &&
+   pod.getStatus.getPhase.toLowerCase(Locale.ROOT) != "running")
   ))
   }
 }


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



[spark] branch master updated: [SPARK-32381][CORE][SQL][FOLLOWUP] More cleanup on HadoopFSUtils

2020-11-18 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 27cd945  [SPARK-32381][CORE][SQL][FOLLOWUP] More cleanup on 
HadoopFSUtils
27cd945 is described below

commit 27cd945c151dccb5ac863e6bc2c4f5b2c6a6d996
Author: Chao Sun 
AuthorDate: Wed Nov 18 12:39:00 2020 -0800

[SPARK-32381][CORE][SQL][FOLLOWUP] More cleanup on HadoopFSUtils

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

This PR is a follow-up of #29471 and does the following improvements for 
`HadoopFSUtils`:
1. Removes the extra `filterFun` from the listing API and combines it with 
the `filter`.
2. Removes `SerializableBlockLocation` and `SerializableFileStatus` given 
that `BlockLocation` and `FileStatus` are already serializable.
3. Hides the `isRootLevel` flag from the top-level API.

### Why are the changes needed?

Main purpose is to simplify the logic within `HadoopFSUtils` as well as 
cleanup the API.

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

No

### How was this patch tested?

Existing unit tests (e.g., `FileIndexSuite`)

Closes #29959 from sunchao/hadoop-fs-utils-followup.

Authored-by: Chao Sun 
Signed-off-by: Holden Karau 
---
 .../org/apache/spark/util/HadoopFSUtils.scala  | 104 -
 .../spark/sql/execution/command/CommandUtils.scala |   2 +-
 .../execution/datasources/InMemoryFileIndex.scala  |  19 ++--
 3 files changed, 31 insertions(+), 94 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala 
b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
index c0a135e..a3a528c 100644
--- a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.viewfs.ViewFileSystem
 import org.apache.hadoop.hdfs.DistributedFileSystem
 
 import org.apache.spark._
-import org.apache.spark.annotation.Private
 import org.apache.spark.internal.Logging
 import org.apache.spark.metrics.source.HiveCatalogMetrics
 
@@ -45,8 +44,6 @@ private[spark] object HadoopFSUtils extends Logging {
* @param paths Input paths to list
* @param hadoopConf Hadoop configuration
* @param filter Path filter used to exclude leaf files from result
-   * @param isRootLevel Whether the input paths are at the root level, i.e., 
they are the root
-   *paths as opposed to nested paths encountered during 
recursive calls of this.
* @param ignoreMissingFiles Ignore missing files that happen during 
recursive listing
*   (e.g., due to race conditions)
* @param ignoreLocality Whether to fetch data locality info when listing 
leaf files. If false,
@@ -57,11 +54,22 @@ private[spark] object HadoopFSUtils extends Logging {
* @param parallelismMax The maximum parallelism for listing. If the number 
of input paths is
*   larger than this value, parallelism will be 
throttled to this value
*   to avoid generating too many tasks.
-   * @param filterFun Optional predicate on the leaf files. Files who failed 
the check will be
-   *  excluded from the results
* @return for each input path, the set of discovered files for the path
*/
   def parallelListLeafFiles(
+sc: SparkContext,
+paths: Seq[Path],
+hadoopConf: Configuration,
+filter: PathFilter,
+ignoreMissingFiles: Boolean,
+ignoreLocality: Boolean,
+parallelismThreshold: Int,
+parallelismMax: Int): Seq[(Path, Seq[FileStatus])] = {
+parallelListLeafFilesInternal(sc, paths, hadoopConf, filter, isRootLevel = 
true,
+  ignoreMissingFiles, ignoreLocality, parallelismThreshold, parallelismMax)
+  }
+
+  private def parallelListLeafFilesInternal(
   sc: SparkContext,
   paths: Seq[Path],
   hadoopConf: Configuration,
@@ -70,8 +78,7 @@ private[spark] object HadoopFSUtils extends Logging {
   ignoreMissingFiles: Boolean,
   ignoreLocality: Boolean,
   parallelismThreshold: Int,
-  parallelismMax: Int,
-  filterFun: Option[String => Boolean] = None): Seq[(Path, 
Seq[FileStatus])] = {
+  parallelismMax: Int): Seq[(Path, Seq[FileStatus])] = {
 
 // Short-circuits parallel listing when serial listing is likely to be 
faster.
 if (paths.size <= parallelismThreshold) {
@@ -85,8 +92,7 @@ private[spark] object HadoopFSUtils extends Logging {
   ignoreLocality = ignoreLocality,
   isRootPath = isRootLevel,
   parallelismThreshold = parallelismThreshold,
-  parallelismMax = parallelismMax,
-  filterFun = filterFun)
+  parallelismMax = parallelismMax)
 (path, leafFiles)
   }
 }
@@ -

[spark] branch branch-3.0 updated: [SPARK-30821][K8S] Handle executor failure with multiple containers

2020-10-24 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new f7c7f4f8 [SPARK-30821][K8S] Handle executor failure with multiple 
containers
f7c7f4f8 is described below

commit f7c7f4f8b6fe7ed55c1aed5494311ba23844d019
Author: Shiqi Sun 
AuthorDate: Sat Oct 24 09:55:57 2020 -0700

[SPARK-30821][K8S] Handle executor failure with multiple containers

Handle executor failure with multiple containers

Added a spark property spark.kubernetes.executor.checkAllContainers,
with default being false. When it's true, the executor snapshot will
take all containers in the executor into consideration when deciding
whether the executor is in "Running" state, if the pod restart policy is
"Never". Also, added the new spark property to the doc.

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

Checking of all containers in the executor pod when reporting executor 
status, if the `spark.kubernetes.executor.checkAllContainers` property is set 
to true.

### Why are the changes needed?

Currently, a pod remains "running" as long as there is at least one running 
container. This prevents Spark from noticing when a container has failed in an 
executor pod with multiple containers. With this change, user can configure the 
behavior to be different. Namely, if any container in the executor pod has 
failed, either the executor process or one of its sidecars, the pod is 
considered to be failed, and it will be rescheduled.

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

Yes, new spark property added.
User is now able to choose whether to turn on this feature using the 
`spark.kubernetes.executor.checkAllContainers` property.

### How was this patch tested?

Unit test was added and all passed.
I tried to run integration test by following the instruction 
[here](https://spark.apache.org/developer-tools.html) (section "Testing K8S") 
and also 
[here](https://github.com/apache/spark/blob/master/resource-managers/kubernetes/integration-tests/README.md),
 but I wasn't able to run it smoothly as it fails to talk with minikube 
cluster. Maybe it's because my minikube version is too new (I'm using 
v1.13.1)...? Since I've been trying it for two days and still can't make it 
work, I decide [...]

Closes #29924 from huskysun/exec-sidecar-failure.

Authored-by: Shiqi Sun 
Signed-off-by: Holden Karau 
(cherry picked from commit f65952772702f0a8772c93b79f562f35c337f5a5)
Signed-off-by: Holden Karau 
---
 docs/running-on-kubernetes.md  |  8 
 .../scala/org/apache/spark/deploy/k8s/Config.scala |  8 
 .../cluster/k8s/ExecutorPodsSnapshot.scala | 16 ++-
 .../cluster/k8s/KubernetesClusterManager.scala |  3 ++
 .../DeterministicExecutorPodsSnapshotsStore.scala  |  2 +
 .../cluster/k8s/ExecutorLifecycleTestUtils.scala   | 32 -
 .../cluster/k8s/ExecutorPodsSnapshotSuite.scala| 56 +++---
 .../k8s/ExecutorPodsSnapshotsStoreSuite.scala  |  1 +
 8 files changed, 108 insertions(+), 18 deletions(-)

diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md
index e6a000d..f49f953 100644
--- a/docs/running-on-kubernetes.md
+++ b/docs/running-on-kubernetes.md
@@ -1158,6 +1158,14 @@ See the [configuration page](configuration.html) for 
information on Spark config
   3.0.0
 
 
+  spark.kubernetes.executor.checkAllContainers
+  false
+  
+  Specify whether executor pods should be check all containers (including 
sidecars) or only the executor container when determining the pod status.
+  
+  3.1.0
+
+
   spark.kubernetes.submission.connectionTimeout
   1
   
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
index 22f4c75..b08875b 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
@@ -409,6 +409,14 @@ private[spark] object Config extends Logging {
   .stringConf
   .createOptional
 
+  val KUBERNETES_EXECUTOR_CHECK_ALL_CONTAINERS =
+ConfigBuilder("spark.kubernetes.executor.checkAllContainers")
+  .doc("If set to true, all containers in the executor pod will be checked 
when reporting" +
+"executor status.")
+  .version("3.1.0")
+  .booleanConf
+  .createWithDefault(false)
+
   val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label."
   val KUBERNETES_DRIVER_ANNOTATION_PREFIX =

[spark] branch master updated: [SPARK-30821][K8S] Handle executor failure with multiple containers

2020-10-24 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 f659527  [SPARK-30821][K8S] Handle executor failure with multiple 
containers
f659527 is described below

commit f65952772702f0a8772c93b79f562f35c337f5a5
Author: Shiqi Sun 
AuthorDate: Sat Oct 24 09:55:57 2020 -0700

[SPARK-30821][K8S] Handle executor failure with multiple containers

Handle executor failure with multiple containers

Added a spark property spark.kubernetes.executor.checkAllContainers,
with default being false. When it's true, the executor snapshot will
take all containers in the executor into consideration when deciding
whether the executor is in "Running" state, if the pod restart policy is
"Never". Also, added the new spark property to the doc.

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

Checking of all containers in the executor pod when reporting executor 
status, if the `spark.kubernetes.executor.checkAllContainers` property is set 
to true.

### Why are the changes needed?

Currently, a pod remains "running" as long as there is at least one running 
container. This prevents Spark from noticing when a container has failed in an 
executor pod with multiple containers. With this change, user can configure the 
behavior to be different. Namely, if any container in the executor pod has 
failed, either the executor process or one of its sidecars, the pod is 
considered to be failed, and it will be rescheduled.

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

Yes, new spark property added.
User is now able to choose whether to turn on this feature using the 
`spark.kubernetes.executor.checkAllContainers` property.

### How was this patch tested?

Unit test was added and all passed.
I tried to run integration test by following the instruction 
[here](https://spark.apache.org/developer-tools.html) (section "Testing K8S") 
and also 
[here](https://github.com/apache/spark/blob/master/resource-managers/kubernetes/integration-tests/README.md),
 but I wasn't able to run it smoothly as it fails to talk with minikube 
cluster. Maybe it's because my minikube version is too new (I'm using 
v1.13.1)...? Since I've been trying it for two days and still can't make it 
work, I decide [...]

Closes #29924 from huskysun/exec-sidecar-failure.

Authored-by: Shiqi Sun 
Signed-off-by: Holden Karau 
---
 docs/running-on-kubernetes.md  |  8 
 .../scala/org/apache/spark/deploy/k8s/Config.scala |  8 
 .../cluster/k8s/ExecutorPodsSnapshot.scala | 16 ++-
 .../cluster/k8s/KubernetesClusterManager.scala |  3 ++
 .../DeterministicExecutorPodsSnapshotsStore.scala  |  2 +
 .../cluster/k8s/ExecutorLifecycleTestUtils.scala   | 32 -
 .../cluster/k8s/ExecutorPodsSnapshotSuite.scala| 56 +++---
 .../k8s/ExecutorPodsSnapshotsStoreSuite.scala  |  1 +
 8 files changed, 108 insertions(+), 18 deletions(-)

diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md
index 3bd1c41..4714e35 100644
--- a/docs/running-on-kubernetes.md
+++ b/docs/running-on-kubernetes.md
@@ -1182,6 +1182,14 @@ See the [configuration page](configuration.html) for 
information on Spark config
   3.0.0
 
 
+  spark.kubernetes.executor.checkAllContainers
+  false
+  
+  Specify whether executor pods should be check all containers (including 
sidecars) or only the executor container when determining the pod status.
+  
+  3.1.0
+
+
   spark.kubernetes.submission.connectionTimeout
   1
   
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
index 00eaff4..d399f66 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
@@ -417,6 +417,14 @@ private[spark] object Config extends Logging {
   .stringConf
   .createOptional
 
+  val KUBERNETES_EXECUTOR_CHECK_ALL_CONTAINERS =
+ConfigBuilder("spark.kubernetes.executor.checkAllContainers")
+  .doc("If set to true, all containers in the executor pod will be checked 
when reporting" +
+"executor status.")
+  .version("3.1.0")
+  .booleanConf
+  .createWithDefault(false)
+
   val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label."
   val KUBERNETES_DRIVER_ANNOTATION_PREFIX = 
"spark.kubernetes.driver.annotation."
   val KUBERNETES_DRIVER_SERVICE_ANNOTATION_PREFIX = 
"spark.kubernetes.driver.servi

[spark] branch master updated: [SPARK-32381][CORE][SQL] Move and refactor parallel listing & non-location sensitive listing to core

2020-09-24 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 8ccfbc1  [SPARK-32381][CORE][SQL] Move and refactor parallel listing & 
non-location sensitive listing to core
8ccfbc1 is described below

commit 8ccfbc114e3e8d9fc919bf05602e02a506566e31
Author: Chao Sun 
AuthorDate: Thu Sep 24 10:58:52 2020 -0700

[SPARK-32381][CORE][SQL] Move and refactor parallel listing & non-location 
sensitive listing to core



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


This moves and refactors the parallel listing utilities from 
`InMemoryFileIndex` to Spark core so it can be reused by modules beside SQL. 
Along the process this also did some cleanups/refactorings:

- Created a `HadoopFSUtils` class under core
- Moved `InMemoryFileIndex.bulkListLeafFiles` into 
`HadoopFSUtils.parallelListLeafFiles`. It now depends on a `SparkContext` 
instead of `SparkSession` in SQL. Also added a few parameters which used to be 
read from `SparkSession.conf`: `ignoreMissingFiles`, `ignoreLocality`, 
`parallelismThreshold`, `parallelismMax ` and `filterFun` (for additional 
filtering support but we may be able to merge this with `filter` parameter in 
future).
- Moved `InMemoryFileIndex.listLeafFiles` into 
`HadoopFSUtils.listLeafFiles` with similar changes above.

### Why are the changes needed?


Currently the locality-aware parallel listing mechanism only applies to 
`InMemoryFileIndex`. By moving this to core, we can potentially reuse the same 
mechanism for other code paths as well.

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


No.

### How was this patch tested?


Since this is mostly a refactoring, it relies on existing unit tests such 
as those for `InMemoryFileIndex`.

Closes #29471 from sunchao/SPARK-32381.

Lead-authored-by: Chao Sun 
Co-authored-by: Holden Karau 
Co-authored-by: Chao Sun 
Signed-off-by: Holden Karau 
---
 .../org/apache/spark/util/HadoopFSUtils.scala  | 352 -
 .../spark/sql/execution/command/CommandUtils.scala |   2 +-
 .../execution/datasources/InMemoryFileIndex.scala  | 297 +
 3 files changed, 150 insertions(+), 501 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
 b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
similarity index 51%
copy from 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
copy to core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
index a488ed1..c0a135e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala
+++ b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.execution.datasources
+package org.apache.spark.util
 
 import java.io.FileNotFoundException
 
@@ -25,175 +25,68 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs._
 import org.apache.hadoop.fs.viewfs.ViewFileSystem
 import org.apache.hadoop.hdfs.DistributedFileSystem
-import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
 
-import org.apache.spark.SparkContext
+import org.apache.spark._
+import org.apache.spark.annotation.Private
 import org.apache.spark.internal.Logging
 import org.apache.spark.metrics.source.HiveCatalogMetrics
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.execution.streaming.FileStreamSink
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.util.SerializableConfiguration
-
 
 /**
- * A [[FileIndex]] that generates the list of files to process by recursively 
listing all the
- * files present in `paths`.
- *
- * @param rootPathsSpecified the list of root table paths to scan (some of 
which might be
- *   filtered out later)
- * @param parameters as set of options to control discovery
- * @param userSpecifiedSchema an optional user specified schema that will be 
use to provide
- *types for the discovered partitions
+ * Utility functions to simplify and speed-up file listing.
  */
-class InMemoryFileIndex(
-sparkSession: SparkSession,
-rootPathsSpecified: Seq[Path],
-parameters: Map[String, String],
-userSpecifiedSchema: Option[StructType],
-fileStatusCache: FileStatusCache = NoopCache,
-userSpecifiedPartitionSpec: Option[PartitionSpec] = None,
-override val metadataOpsTimeNs: Option[Long] = None)
-  extends PartitioningAwareFileIndex(
-sparkSession, parameters, userSpecifiedSchema, fileStatusCache) {
-
-  // Filter out streaming metadata dirs or files such as 
"/.../_s

[spark] branch master updated: [SPARK-32643][CORE][K8S] Consolidate state decommissioning in the TaskSchedulerImpl realm

2020-08-26 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 b786f31  [SPARK-32643][CORE][K8S] Consolidate state decommissioning in 
the TaskSchedulerImpl realm
b786f31 is described below

commit b786f31a42180523b0baa8113e26b2ddee445498
Author: Devesh Agrawal 
AuthorDate: Wed Aug 26 15:16:47 2020 -0700

[SPARK-32643][CORE][K8S] Consolidate state decommissioning in the 
TaskSchedulerImpl realm

### What changes were proposed in this pull request?
The decommissioning state is a bit fragment across two places in the 
TaskSchedulerImpl:

https://github.com/apache/spark/pull/29014/ stored the incoming 
decommission info messages in TaskSchedulerImpl.executorsPendingDecommission.
While https://github.com/apache/spark/pull/28619/ was storing just the 
executor end time in the map TaskSetManager.tidToExecutorKillTimeMapping (which 
in turn is contained in TaskSchedulerImpl).
While the two states are not really overlapping, it's a bit of a code 
hygiene concern to save this state in two places.

With https://github.com/apache/spark/pull/29422, TaskSchedulerImpl is 
emerging as the place where all decommissioning book keeping is kept within the 
driver. So consolidate the information in _tidToExecutorKillTimeMapping_ into 
_executorsPendingDecommission_.

However, in order to do so, we need to walk away from keeping the raw 
ExecutorDecommissionInfo messages and instead keep another class 
ExecutorDecommissionState. This decoupling will allow the RPC message class 
ExecutorDecommissionInfo to evolve independently from the book keeping 
ExecutorDecommissionState.

### Why are the changes needed?

This is just a code cleanup. These two features were added independently 
and its time to consolidate their state for good hygiene.

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

No

### How was this patch tested?

Existing tests.

Closes #29452 from agrawaldevesh/consolidate_decom_state.

Authored-by: Devesh Agrawal 
Signed-off-by: Holden Karau 
---
 .../apache/spark/ExecutorAllocationClient.scala|   2 +-
 .../org/apache/spark/scheduler/DAGScheduler.scala  |   2 +-
 .../spark/scheduler/ExecutorDecommissionInfo.scala |  14 ++-
 .../org/apache/spark/scheduler/TaskScheduler.scala |   2 +-
 .../apache/spark/scheduler/TaskSchedulerImpl.scala |  52 ++
 .../apache/spark/scheduler/TaskSetManager.scala|  34 +++
 .../apache/spark/scheduler/DAGSchedulerSuite.scala |   8 +-
 .../scheduler/ExternalClusterManagerSuite.scala|   4 +-
 .../spark/scheduler/TaskSchedulerImplSuite.scala   | 112 ++---
 .../spark/scheduler/TaskSetManagerSuite.scala  |  33 +++---
 10 files changed, 167 insertions(+), 96 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
index 079340a..ce47f3f 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -88,7 +88,7 @@ private[spark] trait ExecutorAllocationClient {
* Default implementation delegates to kill, scheduler must override
* if it supports graceful decommissioning.
*
-   * @param executorsAndDecominfo identifiers of executors & decom info.
+   * @param executorsAndDecomInfo identifiers of executors & decom info.
* @param adjustTargetNumExecutors whether the target number of executors 
will be adjusted down
* after these executors have been 
decommissioned.
* @return the ids of the executors acknowledged by the cluster manager to 
be removed.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index ae0387e..18cd241 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -1825,7 +1825,7 @@ private[spark] class DAGScheduler(
   if (bmAddress != null) {
 val externalShuffleServiceEnabled = 
env.blockManager.externalShuffleServiceEnabled
 val isHostDecommissioned = taskScheduler
-  .getExecutorDecommissionInfo(bmAddress.executorId)
+  .getExecutorDecommissionState(bmAddress.executorId)
   .exists(_.isHostDecommissioned)
 
 // Shuffle output of all executors on host `bmAddress.host` may be 
lost if:
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala 
b/core/src/main/scala/org/apache/spark/scheduler/ExecutorDecommissionInfo.scala
index a82b5d3..48ae879 100644
--- 
a/core/src/ma

[spark-website] branch asf-site updated: Update committer guide

2020-08-24 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new a3f618b  Update committer guide
a3f618b is described below

commit a3f618bec99e5b4132c6586f12c1313b34cf5b13
Author: Holden Karau 
AuthorDate: Mon Aug 24 10:59:16 2020 -0700

Update committer guide

Update the committer guide with the new policy we voted on for when to 
commit.

Author: Holden Karau 

Closes #284 from holdenk/update-committer-guide.
---
 committers.md| 21 +
 site/committers.html | 42 +-
 2 files changed, 50 insertions(+), 13 deletions(-)

diff --git a/committers.md b/committers.md
index 2665e0a..fc6958c 100644
--- a/committers.md
+++ b/committers.md
@@ -132,6 +132,27 @@ In particular, if you are working on an area of the 
codebase you are unfamiliar
 Git history for that code to see who reviewed patches before. You can do this 
using 
 `git log --format=full `, by examining the "Commit" field to see who 
committed each patch.
 
+When to commit/merge a pull request
+
+PRs shall not be merged during active, on-topic discussion unless they address 
issues such as critical security fixes of a public vulnerability. Under 
extenuating circumstances, PRs may be merged during active, off-topic 
discussion and the discussion directed to a more appropriate venue. Time should 
be given prior to merging for those involved with the conversation to explain 
if they believe they are on-topic.
+
+Lazy consensus requires giving time for discussion to settle while 
understanding that people may not be working on Spark as their full-time job 
and may take holidays. It is believed that by doing this, we can limit how 
often people feel the need to exercise their veto.
+
+
+All -1s with justification merit discussion.  A -1 from a non-committer can be 
overridden only with input from multiple committers, and suitable time must be 
offered for any committer to raise concerns. A -1 from a committer who cannot 
be reached requires a consensus vote of the PMC under ASF voting rules to 
determine the next steps within the [ASF guidelines for code 
vetoes](https://www.apache.org/foundation/voting.html).
+
+
+These policies serve to reiterate the core principle that code must not be 
merged with a pending veto or before a consensus has been reached (lazy or 
otherwise).
+
+
+It is the PMC’s hope that vetoes continue to be infrequent, and when they 
occur, that all parties will take the time to build consensus prior to 
additional feature work.
+
+
+Being a committer means exercising your judgement while working in a community 
of people with diverse views. There is nothing wrong in getting a second (or 
third or fourth) opinion when you are uncertain. Thank you for your dedication 
to the Spark project; it is appreciated by the developers and users of Spark.
+
+
+It is hoped that these guidelines do not slow down development; rather, by 
removing some of the uncertainty, the goal is to make it easier for us to reach 
consensus. If you have ideas on how to improve these guidelines or other Spark 
project operating procedures, you should reach out on the dev@ list to start 
the discussion.
+
 How to Merge a Pull Request
 
 Changes pushed to the master branch on Apache cannot be removed; that is, we 
can't force-push to 
diff --git a/site/committers.html b/site/committers.html
index 91bc57b..ff09913 100644
--- a/site/committers.html
+++ b/site/committers.html
@@ -565,7 +565,23 @@ who have shown they understand and can help with these 
activities.
 Contributing to Spark. 
 In particular, if you are working on an area of the codebase you are 
unfamiliar with, look at the 
 Git history for that code to see who reviewed patches before. You can do this 
using 
-git log --format=full filename, 
by examining the Commit field to see who committed each patch.
+git log --format=full 
filename, by examining the Commit field to see who 
committed each patch.
+
+When to commit/merge a pull request
+
+PRs shall not be merged during active, on-topic discussion unless they 
address issues such as critical security fixes of a public vulnerability. Under 
extenuating circumstances, PRs may be merged during active, off-topic 
discussion and the discussion directed to a more appropriate venue. Time should 
be given prior to merging for those involved with the conversation to explain 
if they believe they are on-topic.
+
+Lazy consensus requires giving time for discussion to settle while 
understanding that people may not be working on Spark as their full-time job 
and may take holidays. It is believed that by doing this, we can limit how 
often people feel the need to exercise their veto.
+
+All -1s with justification merit discussion.  A -1 from a non-committer can 
be overr

[spark] branch master updated: [SPARK-31198][CORE] Use graceful decommissioning as part of dynamic scaling

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

holden 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 548ac7c  [SPARK-31198][CORE] Use graceful decommissioning as part of 
dynamic scaling
548ac7c is described below

commit 548ac7c4af2270a6bdbf7a6f29f4846eecdc0171
Author: Holden Karau 
AuthorDate: Wed Aug 12 17:07:18 2020 -0700

[SPARK-31198][CORE] Use graceful decommissioning as part of dynamic scaling

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

If graceful decommissioning is enabled, Spark's dynamic scaling uses this 
instead of directly killing executors.

### Why are the changes needed?

When scaling down Spark we should avoid triggering recomputes as much as 
possible.

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

Hopefully their jobs run faster or at the same speed. It also enables 
experimental shuffle service free dynamic scaling when graceful decommissioning 
is enabled (using the same code as the shuffle tracking dynamic scaling).

### How was this patch tested?

For now I've extended the ExecutorAllocationManagerSuite for both core & 
streaming.

Closes #29367 from 
holdenk/SPARK-31198-use-graceful-decommissioning-as-part-of-dynamic-scaling.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../apache/spark/ExecutorAllocationClient.scala|  40 +
 .../apache/spark/ExecutorAllocationManager.scala   |  30 +++-
 .../cluster/CoarseGrainedSchedulerBackend.scala| 190 -
 .../cluster/StandaloneSchedulerBackend.scala   |   3 +-
 .../spark/scheduler/dynalloc/ExecutorMonitor.scala |  61 ++-
 .../org/apache/spark/storage/BlockManager.scala|   2 +-
 .../apache/spark/storage/BlockManagerMaster.scala  |   6 +-
 .../spark/ExecutorAllocationManagerSuite.scala |  71 +++-
 .../WorkerDecommissionExtendedSuite.scala  |   3 +-
 .../spark/scheduler/WorkerDecommissionSuite.scala  |   4 +-
 .../BlockManagerDecommissionIntegrationSuite.scala |   7 +-
 project/SparkBuild.scala   |   2 +
 .../docker/src/main/dockerfiles/spark/decom.sh |   2 +-
 .../k8s/integrationtest/KubernetesSuite.scala  |  27 ++-
 .../integration-tests/tests/decommissioning.py |   5 -
 .../scheduler/ExecutorAllocationManager.scala  |  10 +-
 .../scheduler/ExecutorAllocationManagerSuite.scala |  51 --
 17 files changed, 380 insertions(+), 134 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala 
b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
index 00bd006..079340a 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark
 
+import org.apache.spark.scheduler.ExecutorDecommissionInfo
+
 /**
  * A client that communicates with the cluster manager to request or kill 
executors.
  * This is currently supported only in YARN mode.
@@ -82,6 +84,44 @@ private[spark] trait ExecutorAllocationClient {
 force: Boolean = false): Seq[String]
 
   /**
+   * Request that the cluster manager decommission the specified executors.
+   * Default implementation delegates to kill, scheduler must override
+   * if it supports graceful decommissioning.
+   *
+   * @param executorsAndDecominfo identifiers of executors & decom info.
+   * @param adjustTargetNumExecutors whether the target number of executors 
will be adjusted down
+   * after these executors have been 
decommissioned.
+   * @return the ids of the executors acknowledged by the cluster manager to 
be removed.
+   */
+  def decommissionExecutors(
+executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
+adjustTargetNumExecutors: Boolean): Seq[String] = {
+killExecutors(executorsAndDecomInfo.map(_._1),
+  adjustTargetNumExecutors,
+  countFailures = false)
+  }
+
+
+  /**
+   * Request that the cluster manager decommission the specified executor.
+   * Delegates to decommissionExecutors.
+   *
+   * @param executorId identifiers of executor to decommission
+   * @param decommissionInfo information about the decommission (reason, host 
loss)
+   * @param adjustTargetNumExecutors if we should adjust the target number of 
executors.
+   * @return whether the request is acknowledged by the cluster manager.
+   */
+  final def decommissionExecutor(executorId: String,
+  decommissionInfo: ExecutorDecommissionInfo,
+  adjustTargetNumExecutors: Boolean): Boolean = {
+val decommissionedExecutors = decommissionExecutors(
+  Array((executorId, decommissionInfo)),
+  adjustTargetNumExecutors = adjustTargetNumE

[spark] branch master updated: [SPARK-31197][CORE] Shutdown executor once we are done decommissioning

2020-08-05 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 375d348  [SPARK-31197][CORE] Shutdown executor once we are done 
decommissioning
375d348 is described below

commit 375d348a83e6ffa38dfaece5047633f67aee1da5
Author: Holden Karau 
AuthorDate: Wed Aug 5 16:28:14 2020 -0700

[SPARK-31197][CORE] Shutdown executor once we are done decommissioning

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

Exit the executor when it has been asked to decommission and there is 
nothing left for it to do.

This is a rebase of https://github.com/apache/spark/pull/28817

### Why are the changes needed?

If we want to use decommissioning in Spark's own scale down we should 
terminate the executor once finished.
Furthermore, in graceful shutdown it makes sense to release resources we no 
longer need if we've been asked to shutdown by the cluster manager instead of 
always holding the resources as long as possible.

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

The decommissioned executors will exit and the end of decommissioning. This 
is sort of a user facing change, however decommissioning hasn't been in any 
releases yet.

### How was this patch tested?

I changed the unit test to not send the executor exit message and still 
wait on the executor exited message.

Closes #29211 from holdenk/SPARK-31197-exit-execs-redone.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 .../org/apache/spark/deploy/DeployMessage.scala|   2 -
 .../org/apache/spark/deploy/worker/Worker.scala|   2 +-
 .../executor/CoarseGrainedExecutorBackend.scala|  58 -
 .../cluster/CoarseGrainedClusterMessage.scala  |   3 +
 .../cluster/CoarseGrainedSchedulerBackend.scala|  10 ++
 .../org/apache/spark/storage/BlockManager.scala|   8 ++
 .../spark/storage/BlockManagerDecommissioner.scala |  96 +++---
 .../spark/scheduler/WorkerDecommissionSuite.scala  |  19 ++-
 .../BlockManagerDecommissionIntegrationSuite.scala |  17 ++-
 .../BlockManagerDecommissionUnitSuite.scala| 139 -
 10 files changed, 310 insertions(+), 44 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala 
b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index c8c6e5a..b7a64d75 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -165,8 +165,6 @@ private[deploy] object DeployMessages {
 
   case object ReregisterWithMaster // used when a worker attempts to reconnect 
to a master
 
-  case object DecommissionSelf // Mark as decommissioned. May be Master to 
Worker in the future.
-
   // AppClient to Master
 
   case class RegisterApplication(appDescription: ApplicationDescription, 
driver: RpcEndpointRef)
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 aa8c46f..862e685 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
@@ -668,7 +668,7 @@ private[deploy] class Worker(
   finishedApps += id
   maybeCleanupApplication(id)
 
-case DecommissionSelf =>
+case WorkerDecommission(_, _) =>
   decommissionSelf()
   }
 
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 def125b..55fb76b 100644
--- 
a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ 
b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -64,7 +64,6 @@ private[spark] class CoarseGrainedExecutorBackend(
 
   private[this] val stopping = new AtomicBoolean(false)
   var executor: Executor = null
-  @volatile private var decommissioned = false
   @volatile var driver: Option[RpcEndpointRef] = None
 
   // If this CoarseGrainedExecutorBackend is changed to support multiple 
threads, then this may need
@@ -80,6 +79,8 @@ private[spark] class CoarseGrainedExecutorBackend(
*/
   private[executor] val taskResources = new mutable.HashMap[Long, Map[String, 
ResourceInformation]]
 
+  @volatile private var decommissioned = false
+
   override def onStart(): Unit = {
 logInfo("Registering PWR handler.")
 SignalUtils.register("PWR", "Failed to register SIGPWR handler - " +
@@ -214,6 +215,10 @@ private[spark] class CoarseGrainedExecutorBackend(
 case UpdateDelegationTokens(tokenBytes) =>
   logInfo(s"Received tokens of ${tokenBytes.length} bytes"

[spark] branch master updated: [SPARK-32417] Fix flakyness of BlockManagerDecommissionIntegrationSuite

2020-07-30 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 6032c5b  [SPARK-32417] Fix flakyness of 
BlockManagerDecommissionIntegrationSuite
6032c5b is described below

commit 6032c5b0320fe70455586f4ce863d5d9361b5e07
Author: Devesh Agrawal 
AuthorDate: Thu Jul 30 12:00:19 2020 -0700

[SPARK-32417] Fix flakyness of BlockManagerDecommissionIntegrationSuite

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

This test tries to fix the flakyness of 
BlockManagerDecommissionIntegrationSuite.

### Description of the problem

Make the block manager decommissioning test be less flaky

An interesting failure happens when migrateDuring = true (and persist or 
shuffle is true):
- We schedule the job with tasks on executors 0, 1, 2.
- We wait 300 ms and decommission executor 0.
- If the task is not yet done on executor 0, it will now fail because
   the block manager won't be able to save the block. This condition is
   easy to trigger on a loaded machine where the github checks run.
- The task with retry on a different executor (1 or 2) and its shuffle
   blocks will land there.
- No actual block migration happens here because the decommissioned
   executor technically failed before it could even produce a block.

To remove the above race, this change replaces the fixed wait for 300 ms to 
wait for an actual task to succeed. When a task has succeeded, we know its 
blocks would have been written for sure and thus its executor would certainly 
be forced to migrate those blocks when it is decommissioned.

The change always decommissions an executor on which a real task finished 
successfully instead of picking the first executor. Because the system may 
choose to schedule nothing on the first executor and instead run the two tasks 
on one executor.

### Why are the changes needed?

I have had bad luck with BlockManagerDecommissionIntegrationSuite and it 
has failed several times on my PRs. So fixing it.

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

No, unit test only change.

### How was this patch tested?

Github checks. Ran this test 100 times, 10 at a time in parallel in a 
script.

Closes #29226 from agrawaldevesh/block-manager-decom-flaky.

Authored-by: Devesh Agrawal 
Signed-off-by: Holden Karau 
---
 .../BlockManagerDecommissionIntegrationSuite.scala | 147 +++--
 1 file changed, 103 insertions(+), 44 deletions(-)

diff --git 
a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala
 
b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala
index 5741010..6a52f72 100644
--- 
a/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/storage/BlockManagerDecommissionIntegrationSuite.scala
@@ -17,8 +17,9 @@
 
 package org.apache.spark.storage
 
-import java.util.concurrent.Semaphore
+import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue, 
Semaphore}
 
+import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 import scala.concurrent.duration._
 
@@ -28,29 +29,40 @@ import org.apache.spark._
 import org.apache.spark.internal.config
 import org.apache.spark.scheduler._
 import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend
-import org.apache.spark.util.{ResetSystemProperties, ThreadUtils}
+import org.apache.spark.util.{ResetSystemProperties, SystemClock, ThreadUtils}
 
 class BlockManagerDecommissionIntegrationSuite extends SparkFunSuite with 
LocalSparkContext
 with ResetSystemProperties with Eventually {
 
   val numExecs = 3
   val numParts = 3
+  val TaskStarted = "TASK_STARTED"
+  val TaskEnded = "TASK_ENDED"
+  val JobEnded = "JOB_ENDED"
 
   test(s"verify that an already running task which is going to cache data 
succeeds " +
-s"on a decommissioned executor") {
-runDecomTest(true, false, true)
+s"on a decommissioned executor after task start") {
+runDecomTest(true, false, TaskStarted)
   }
 
-  test(s"verify that shuffle blocks are migrated") {
-runDecomTest(false, true, false)
+  test(s"verify that an already running task which is going to cache data 
succeeds " +
+s"on a decommissioned executor after one task ends but before job ends") {
+runDecomTest(true, false, TaskEnded)
   }
 
-  test(s"verify that both migrations can work at the same time.") {
-runDecomTest(true, true, false)
+  test(s"verify that shuffle blocks are migrated") {
+runDecomTest(false, true, JobEnded)
   }
 

[spark] branch master updated: [SPARK-32199][SPARK-32198] Reduce job failures during decommissioning

2020-07-30 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 366a178  [SPARK-32199][SPARK-32198] Reduce job failures during 
decommissioning
366a178 is described below

commit 366a1789333bac97643159857a206bcd773761a4
Author: Devesh Agrawal 
AuthorDate: Thu Jul 30 11:58:11 2020 -0700

[SPARK-32199][SPARK-32198] Reduce job failures during decommissioning

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

This PR reduces the prospect of a job loss during decommissioning. It
fixes two holes in the current decommissioning framework:

- (a) Loss of decommissioned executors is not treated as a job failure:
We know that the decommissioned executor would be dying soon, so its death 
is
clearly not caused by the application.

- (b) Shuffle files on the decommissioned host are cleared when the
first fetch failure is detected from a decommissioned host: This is a
bit tricky in terms of when to clear the shuffle state ? Ideally you
want to clear it the millisecond before the shuffle service on the node
dies (or the executor dies when there is no external shuffle service) --
too soon and it could lead to some wastage and too late would lead to
fetch failures.

  The approach here is to do this clearing when the very first fetch
failure is observed on the decommissioned block manager, without waiting for
other blocks to also signal a failure.

### Why are the changes needed?

Without them decommissioning a lot of executors at a time leads to job 
failures.

### Code overview

The task scheduler tracks the executors that were decommissioned along with 
their
`ExecutorDecommissionInfo`. This information is used by: (a) For handling a 
`ExecutorProcessLost` error, or (b) by the `DAGScheduler` when handling a fetch 
failure.

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

No

### How was this patch tested?

Added a new unit test `DecommissionWorkerSuite` to test the new behavior by 
exercising the Master-Worker decommissioning. I chose to add a new test since 
the setup logic was quite different from the existing 
`WorkerDecommissionSuite`. I am open to changing the name of the newly added 
test suite :-)

### Questions for reviewers
- Should I add a feature flag to guard these two behaviors ? They seem safe 
to me that they should only get triggered by decommissioning, but you never 
know :-).

Closes #29014 from agrawaldevesh/decom_harden.

Authored-by: Devesh Agrawal 
Signed-off-by: Holden Karau 
---
 .../org/apache/spark/scheduler/DAGScheduler.scala  |  19 +-
 .../spark/scheduler/ExecutorLossReason.scala   |   7 +-
 .../org/apache/spark/scheduler/TaskScheduler.scala |   5 +
 .../apache/spark/scheduler/TaskSchedulerImpl.scala |  37 +-
 .../apache/spark/scheduler/TaskSetManager.scala|   1 +
 .../spark/deploy/DecommissionWorkerSuite.scala | 424 +
 .../apache/spark/scheduler/DAGSchedulerSuite.scala |   4 +
 .../scheduler/ExternalClusterManagerSuite.scala|   2 +
 .../spark/scheduler/TaskSchedulerImplSuite.scala   |  47 +++
 9 files changed, 539 insertions(+), 7 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 2503ae0..6b376cd 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -1821,10 +1821,19 @@ private[spark] class DAGScheduler(
 
   // TODO: mark the executor as failed only if there were lots of 
fetch failures on it
   if (bmAddress != null) {
-val hostToUnregisterOutputs = if 
(env.blockManager.externalShuffleServiceEnabled &&
-  unRegisterOutputOnHostOnFetchFailure) {
-  // We had a fetch failure with the external shuffle service, so 
we
-  // assume all shuffle data on the node is bad.
+val externalShuffleServiceEnabled = 
env.blockManager.externalShuffleServiceEnabled
+val isHostDecommissioned = taskScheduler
+  .getExecutorDecommissionInfo(bmAddress.executorId)
+  .exists(_.isHostDecommissioned)
+
+// Shuffle output of all executors on host `bmAddress.host` may be 
lost if:
+// - External shuffle service is enabled, so we assume that all 
shuffle data on node is
+//   bad.
+// - Host is decommissioned, thus all executors on that host will 
die.
+val shuffleOutputOfEntireHostLost = externalShuffleServiceEnabled 
||
+  isHostDecommissioned
+val hostToUnregiste

[spark] branch master updated: [SPARK-32217] Plumb whether a worker would also be decommissioned along with executor

2020-07-22 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 f8d29d3  [SPARK-32217] Plumb whether a worker would also be 
decommissioned along with executor
f8d29d3 is described below

commit f8d29d371cdf0b8be6a48a9124ffbc3c0794f32a
Author: Devesh Agrawal 
AuthorDate: Wed Jul 22 21:04:06 2020 -0700

[SPARK-32217] Plumb whether a worker would also be decommissioned along 
with executor

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

This PR is a giant plumbing PR that plumbs an `ExecutorDecommissionInfo` 
along
with the DecommissionExecutor message.

### Why are the changes needed?

The primary motivation is to know whether a decommissioned executor
would also be loosing shuffle files -- and thus it is important to know
whether the host would also be decommissioned.

In the absence of this PR, the existing code assumes that decommissioning 
an executor does not loose the whole host with it, and thus does not clear the 
shuffle state if external shuffle service is enabled. While this may hold in 
some cases (like K8s decommissioning an executor pod, or YARN container 
preemption), it does not hold in others like when the cluster is managed by a 
Standalone Scheduler (Master). This is similar to the existing `workerLost` 
field in the `ExecutorProcessLost [...]

In the future, this `ExecutorDecommissionInfo` can be embellished for
knowing how long the executor has to live for scenarios like Cloud spot
kills (or Yarn preemption) and the like.

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

### How was this patch tested?
Tweaked an existing unit test in `AppClientSuite`

Closes #29032 from agrawaldevesh/plumb_decom_info.

Authored-by: Devesh Agrawal 
Signed-off-by: Holden Karau 
---
 .../spark/deploy/client/StandaloneAppClient.scala  |  4 +++-
 .../client/StandaloneAppClientListener.scala   |  4 +++-
 .../org/apache/spark/deploy/master/Master.scala|  5 +++-
 .../executor/CoarseGrainedExecutorBackend.scala| 16 +
 .../spark/scheduler/ExecutorDecommissionInfo.scala | 28 ++
 .../org/apache/spark/scheduler/TaskScheduler.scala |  2 +-
 .../apache/spark/scheduler/TaskSchedulerImpl.scala |  3 ++-
 .../cluster/CoarseGrainedClusterMessage.scala  |  4 +++-
 .../cluster/CoarseGrainedSchedulerBackend.scala| 22 +
 .../cluster/StandaloneSchedulerBackend.scala   |  6 ++---
 .../spark/deploy/client/AppClientSuite.scala   | 15 
 .../apache/spark/scheduler/DAGSchedulerSuite.scala |  8 +--
 .../scheduler/ExternalClusterManagerSuite.scala|  4 +++-
 .../WorkerDecommissionExtendedSuite.scala  |  2 +-
 .../spark/scheduler/WorkerDecommissionSuite.scala  |  2 +-
 .../BlockManagerDecommissionIntegrationSuite.scala |  2 +-
 16 files changed, 92 insertions(+), 35 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala 
b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala
index eedf5e9..a6da839 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala
@@ -31,6 +31,7 @@ import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.Master
 import org.apache.spark.internal.Logging
 import org.apache.spark.rpc._
+import org.apache.spark.scheduler.ExecutorDecommissionInfo
 import org.apache.spark.util.{RpcUtils, ThreadUtils}
 
 /**
@@ -181,7 +182,8 @@ private[spark] class StandaloneAppClient(
 if (ExecutorState.isFinished(state)) {
   listener.executorRemoved(fullId, message.getOrElse(""), exitStatus, 
workerLost)
 } else if (state == ExecutorState.DECOMMISSIONED) {
-  listener.executorDecommissioned(fullId, message.getOrElse(""))
+  listener.executorDecommissioned(fullId,
+ExecutorDecommissionInfo(message.getOrElse(""), 
isHostDecommissioned = workerLost))
 }
 
   case WorkerRemoved(id, host, message) =>
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala
 
b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala
index 2e38a68..e72f7e9 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.deploy.client
 
+import org.apache.spark.scheduler.ExecutorDecommissionInfo
+
 /**
  * Callbacks invoked by deploy client when various events happen. There are 
curr

[spark] branch master updated: [SPARK-24266][K8S] Restart the watcher when we receive a version changed from k8s

2020-07-21 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 0432379  [SPARK-24266][K8S] Restart the watcher when we receive a 
version changed from k8s
0432379 is described below

commit 0432379f9923768a767566e9ac5a4021cfe8d052
Author: Stijn De Haes 
AuthorDate: Tue Jul 21 16:34:30 2020 -0700

[SPARK-24266][K8S] Restart the watcher when we receive a version changed 
from k8s

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

Restart the watcher when it failed with a HTTP_GONE code from the 
kubernetes api. Which means a resource version has changed.

For more relevant information see here: 
https://github.com/fabric8io/kubernetes-client/issues/1075

### Why are the changes needed?

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

No

### How was this patch tested?

Running spark-submit to a k8s cluster.

Not sure how to make an automated test for this. If someone can help me out 
that would be great.

Closes #28423 from stijndehaes/bugfix/k8s-submit-resource-version-change.

Authored-by: Stijn De Haes 
Signed-off-by: Holden Karau 
---
 .../k8s/submit/KubernetesClientApplication.scala   | 52 ++
 .../k8s/submit/LoggingPodStatusWatcher.scala   | 35 +++
 .../spark/deploy/k8s/submit/ClientSuite.scala  |  1 +
 3 files changed, 61 insertions(+), 27 deletions(-)

diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
index b4155fe..713d35d 100644
--- 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
+++ 
b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala
@@ -21,9 +21,11 @@ import java.util.{Collections, UUID}
 import java.util.Properties
 
 import io.fabric8.kubernetes.api.model._
-import io.fabric8.kubernetes.client.KubernetesClient
+import io.fabric8.kubernetes.client.{KubernetesClient, Watch}
+import io.fabric8.kubernetes.client.Watcher.Action
 import scala.collection.mutable
 import scala.util.control.NonFatal
+import util.control.Breaks._
 
 import org.apache.spark.SparkConf
 import org.apache.spark.deploy.SparkApplication
@@ -127,25 +129,37 @@ private[spark] class Client(
 .endSpec()
   .build()
 val driverPodName = resolvedDriverPod.getMetadata.getName
-Utils.tryWithResource(
-  kubernetesClient
-.pods()
-.withName(driverPodName)
-.watch(watcher)) { _ =>
-  val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod)
-  try {
-val otherKubernetesResources =
-  resolvedDriverSpec.driverKubernetesResources ++ Seq(configMap)
-addDriverOwnerReference(createdDriverPod, otherKubernetesResources)
-kubernetesClient.resourceList(otherKubernetesResources: 
_*).createOrReplace()
-  } catch {
-case NonFatal(e) =>
-  kubernetesClient.pods().delete(createdDriverPod)
-  throw e
-  }
 
-  val sId = Seq(conf.namespace, driverPodName).mkString(":")
-  watcher.watchOrStop(sId)
+var watch: Watch = null
+val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod)
+try {
+  val otherKubernetesResources = 
resolvedDriverSpec.driverKubernetesResources ++ Seq(configMap)
+  addDriverOwnerReference(createdDriverPod, otherKubernetesResources)
+  kubernetesClient.resourceList(otherKubernetesResources: 
_*).createOrReplace()
+} catch {
+  case NonFatal(e) =>
+kubernetesClient.pods().delete(createdDriverPod)
+throw e
+}
+val sId = Seq(conf.namespace, driverPodName).mkString(":")
+breakable {
+  while (true) {
+val podWithName = kubernetesClient
+  .pods()
+  .withName(driverPodName)
+// Reset resource to old before we start the watch, this is important 
for race conditions
+watcher.reset()
+watch = podWithName.watch(watcher)
+
+// Send the latest pod state we know to the watcher to make sure we 
didn't miss anything
+watcher.eventReceived(Action.MODIFIED, podWithName.get())
+
+// Break the while loop if the pod is completed or we don't want to 
wait
+if(watcher.watchOrStop(sId)) {
+  watch.close()
+  break
+}
+  }
 }
   }
 
diff --git 
a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala
 
b/resource-managers/kubernetes/core/src/main/scala/org/a

[spark] branch master updated: [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

2020-07-19 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 a4ca355  [SPARK-20629][CORE][K8S] Copy shuffle data when nodes are 
being shutdown
a4ca355 is described below

commit a4ca355af8556e8c5948e492ef70ef0b48416dc4
Author: Holden Karau 
AuthorDate: Sun Jul 19 21:33:13 2020 -0700

[SPARK-20629][CORE][K8S] Copy shuffle data when nodes are being shutdown

### What is changed?

This pull request adds the ability to migrate shuffle files during Spark's 
decommissioning. The design document associated with this change is at 
https://docs.google.com/document/d/1xVO1b6KAwdUhjEJBolVPl9C6sLj7oOveErwDSYdT-pE 
.

To allow this change the `MapOutputTracker` has been extended to allow the 
location of shuffle files to be updated with `updateMapOutput`. When a shuffle 
block is put, a block update message will be sent which triggers the 
`updateMapOutput`.

Instead of rejecting remote puts of shuffle blocks `BlockManager` delegates 
the storage of shuffle blocks to it's shufflemanager's resolver (if supported). 
A new, experimental, trait is added for shuffle resolvers to indicate they 
handle remote putting of blocks.

The existing block migration code is moved out into a separate file, and a 
producer/consumer model is introduced for migrating shuffle files from the host 
as quickly as possible while not overwhelming other executors.

### Why are the changes needed?

Recomputting shuffle blocks can be expensive, we should take advantage of 
our decommissioning time to migrate these blocks.

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

This PR introduces two new configs parameters, 
`spark.storage.decommission.shuffleBlocks.enabled` & 
`spark.storage.decommission.rddBlocks.enabled` that control which blocks should 
be migrated during storage decommissioning.

### How was this patch tested?

New unit test & expansion of the Spark on K8s decom test to assert that 
decommisioning with shuffle block migration means that the results are not 
recomputed even when the original executor is terminated.

This PR is a cleaned-up version of the previous WIP PR I made 
https://github.com/apache/spark/pull/28331 (thanks to attilapiros for his very 
helpful reviewing on it :)).

Closes #28708 from 
holdenk/SPARK-20629-copy-shuffle-data-when-nodes-are-being-shutdown-cleaned-up.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Co-authored-by: “attilapiros” 
Co-authored-by: Attila Zsolt Piros 
Signed-off-by: Holden Karau 
---
 .../scala/org/apache/spark/MapOutputTracker.scala  |  38 ++-
 .../src/main/scala/org/apache/spark/SparkEnv.scala |   3 +-
 .../org/apache/spark/internal/config/package.scala |  23 ++
 .../network/netty/NettyBlockTransferService.scala  |   5 +-
 .../org/apache/spark/scheduler/MapStatus.scala |  15 +-
 .../cluster/StandaloneSchedulerBackend.scala   |   2 +-
 .../spark/shuffle/IndexShuffleBlockResolver.scala  |  99 ++-
 .../apache/spark/shuffle/MigratableResolver.scala  |  48 +++
 .../apache/spark/shuffle/ShuffleBlockInfo.scala|  28 ++
 .../scala/org/apache/spark/storage/BlockId.scala   |   5 +-
 .../org/apache/spark/storage/BlockManager.scala| 154 +++---
 .../spark/storage/BlockManagerDecommissioner.scala | 330 +
 .../spark/storage/BlockManagerMasterEndpoint.scala |  26 +-
 ...avedOnDecommissionedBlockManagerException.scala |  21 ++
 .../spark/scheduler/WorkerDecommissionSuite.scala  |   2 +-
 .../sort/IndexShuffleBlockResolverSuite.scala  |   3 +-
 .../org/apache/spark/storage/BlockIdSuite.scala|   4 +-
 .../BlockManagerDecommissionIntegrationSuite.scala | 229 ++
 .../storage/BlockManagerDecommissionSuite.scala| 106 ---
 .../BlockManagerDecommissionUnitSuite.scala|  92 ++
 .../storage/BlockManagerReplicationSuite.scala |   2 +-
 .../apache/spark/storage/BlockManagerSuite.scala   |  93 +-
 .../k8s/integrationtest/DecommissionSuite.scala|  13 +-
 .../k8s/integrationtest/KubernetesSuite.scala  |  35 ++-
 .../integration-tests/tests/decommissioning.py |  27 +-
 .../streaming/ReceivedBlockHandlerSuite.scala  |   2 +-
 26 files changed, 1150 insertions(+), 255 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala 
b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 32251df..64102cc 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -49,7 +49,7 @@ import org.apache.spark.util._
  *
  * All public methods of this class are thread-safe.
  */
-private class ShuffleStatus(numPartitions: Int) {
+private class ShuffleStatus(numPartiti

[spark] branch master updated: [SPARK-21040][CORE] Speculate tasks which are running on decommission executors

2020-07-17 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 0678afe  [SPARK-21040][CORE] Speculate tasks which are running on 
decommission executors
0678afe is described below

commit 0678afe393b1e4f65b70470483fe0cdb1fe139dc
Author: Prakhar Jain 
AuthorDate: Fri Jul 17 16:11:02 2020 -0700

[SPARK-21040][CORE] Speculate tasks which are running on decommission 
executors

### What changes were proposed in this pull request?
This PR adds functionality to consider the running tasks on decommission 
executors based on some config.
In spark-on-cloud , we sometimes already know that an executor won't be 
alive for more than fix amount of time. Ex- In AWS Spot nodes, once we get the 
notification, we know that a node will be gone in 120 seconds.
So if the running tasks on the decommissioning executors may run beyond 
currentTime+120 seconds, then they are candidate for speculation.

### Why are the changes needed?
Currently when an executor is decommission, we stop scheduling new tasks on 
those executors but the already running tasks keeps on running on them. Based 
on the cloud, we might know beforehand that an executor won't be alive for more 
than a preconfigured time. Different cloud providers gives different timeouts 
before they take away the nodes. For Ex- In case of AWS spot nodes, an executor 
won't be alive for more than 120 seconds. We can utilize this information in 
cloud environments a [...]

### Does this PR introduce _any_ user-facing change?
Yes. This PR adds a new config "spark.executor.decommission.killInterval" 
which they can explicitly set based on the cloud environment where they are 
running.

### How was this patch tested?
Added UT.

Closes #28619 from 
prakharjain09/SPARK-21040-speculate-decommission-exec-tasks.

Authored-by: Prakhar Jain 
Signed-off-by: Holden Karau 
---
 .../org/apache/spark/internal/config/package.scala |  11 +++
 .../apache/spark/scheduler/TaskSetManager.scala|  28 +-
 .../spark/scheduler/TaskSetManagerSuite.scala  | 106 +
 3 files changed, 141 insertions(+), 4 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 ca75a19..f0b292b 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
@@ -1843,6 +1843,17 @@ package object config {
   .timeConf(TimeUnit.MILLISECONDS)
   .createOptional
 
+  private[spark] val EXECUTOR_DECOMMISSION_KILL_INTERVAL =
+ConfigBuilder("spark.executor.decommission.killInterval")
+  .doc("Duration after which a decommissioned executor will be killed 
forcefully." +
+"This config is useful for cloud environments where we know in advance 
when " +
+"an executor is going to go down after decommissioning signal i.e. 
around 2 mins " +
+"in aws spot nodes, 1/2 hrs in spot block nodes etc. This config is 
currently " +
+"used to decide what tasks running on decommission executors to 
speculate.")
+  .version("3.1.0")
+  .timeConf(TimeUnit.SECONDS)
+  .createOptional
+
   private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir")
 .doc("Staging directory used while submitting applications.")
 .version("2.0.0")
diff --git 
a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala 
b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index a302f68..4b31ff0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -19,7 +19,7 @@ package org.apache.spark.scheduler
 
 import java.io.NotSerializableException
 import java.nio.ByteBuffer
-import java.util.concurrent.ConcurrentLinkedQueue
+import java.util.concurrent.{ConcurrentLinkedQueue, TimeUnit}
 
 import scala.collection.immutable.Map
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
@@ -102,6 +102,8 @@ private[spark] class TaskSetManager(
 }
 numTasks <= slots
   }
+  val executorDecommissionKillInterval = 
conf.get(EXECUTOR_DECOMMISSION_KILL_INTERVAL).map(
+TimeUnit.SECONDS.toMillis)
 
   // For each task, tracks whether a copy of the task has succeeded. A task 
will also be
   // marked as "succeeded" if it failed with a fetch failure, in which case it 
should not
@@ -165,6 +167,7 @@ private[spark] class TaskSetManager(
 
   // Task index, start and finish time for each task attempt (indexed by task 
ID)
   private[scheduler] val taskInfos

[spark-website] branch asf-site updated: Fix 2-4-6 web build

2020-06-10 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to branch asf-site
in repository https://gitbox.apache.org/repos/asf/spark-website.git


The following commit(s) were added to refs/heads/asf-site by this push:
 new 3d9740f  Fix 2-4-6 web build
3d9740f is described below

commit 3d9740f38beca3b8609b8650409edb93a70c1aec
Author: Holden Karau 
AuthorDate: Wed Jun 10 18:36:12 2020 -0700

Fix 2-4-6 web build

Fix the 2.4.6 web build, the jekyll serve wrote some localhost values in 
the sitemap we don't want and add the generated release files.

Author: Holden Karau 

Closes #266 from holdenk/spark-2-4-6-rebuild.
---
 site/mailing-lists.html|   2 +-
 site/{mailing-lists.html => news/spark-2-4-6.html} |  16 +-
 .../spark-release-2-4-6.html}  |  56 +++-
 site/sitemap.xml   | 370 ++---
 4 files changed, 248 insertions(+), 196 deletions(-)

diff --git a/site/mailing-lists.html b/site/mailing-lists.html
index 2f4a88f..f6686f9 100644
--- a/site/mailing-lists.html
+++ b/site/mailing-lists.html
@@ -12,7 +12,7 @@
 
   
 
-http://localhost:4000/community.html; />
+https://spark.apache.org/community.html; />
   
 
   
diff --git a/site/mailing-lists.html b/site/news/spark-2-4-6.html
similarity index 94%
copy from site/mailing-lists.html
copy to site/news/spark-2-4-6.html
index 2f4a88f..53d1399 100644
--- a/site/mailing-lists.html
+++ b/site/news/spark-2-4-6.html
@@ -6,14 +6,11 @@
   
 
   
- Mailing Lists | Apache Spark
+ Spark 2.4.6 released | Apache Spark
 
   
 
   
-
-http://localhost:4000/community.html; />
-  
 
   
 
@@ -203,7 +200,16 @@
   
 
   
-
+Spark 2.4.6 released
+
+
+We are happy to announce the availability of Spark 
2.4.6! Visit the release notes to read about the new features, or download the release today.
+
+
+
+
+Spark News Archive
+
 
   
 
diff --git a/site/mailing-lists.html b/site/releases/spark-release-2-4-6.html
similarity index 68%
copy from site/mailing-lists.html
copy to site/releases/spark-release-2-4-6.html
index 2f4a88f..299cf58 100644
--- a/site/mailing-lists.html
+++ b/site/releases/spark-release-2-4-6.html
@@ -6,14 +6,11 @@
   
 
   
- Mailing Lists | Apache Spark
+ Spark Release 2.4.6 | Apache Spark
 
   
 
   
-
-http://localhost:4000/community.html; />
-  
 
   
 
@@ -203,7 +200,56 @@
   
 
   
-
+Spark Release 2.4.6
+
+
+Spark 2.4.6 is a maintenance release containing stability, correctness, and 
security fixes. This release is based on the branch-2.4 maintenance branch of 
Spark. We strongly recommend all 2.4 users to upgrade to this stable 
release.
+
+Notable changes
+
+  https://issues.apache.org/jira/browse/SPARK-29419;>[SPARK-29419]: 
Seq.toDS / spark.createDataset(Seq) is not thread-safe
+  https://issues.apache.org/jira/browse/SPARK-31519;>[SPARK-31519]: 
Cast in having aggregate expressions returns the wrong result
+  https://issues.apache.org/jira/browse/SPARK-26293;>[SPARK-26293]: 
Cast exception when having python udf in subquery
+  https://issues.apache.org/jira/browse/SPARK-30826;>[SPARK-30826]: 
LIKE returns wrong result from external table using parquet
+  https://issues.apache.org/jira/browse/SPARK-30857;>[SPARK-30857]: 
Wrong truncations of timestamps before the epoch to hours and days
+  https://issues.apache.org/jira/browse/SPARK-31256;>[SPARK-31256]: 
Dropna doesnt work for struct columns
+  https://issues.apache.org/jira/browse/SPARK-31312;>[SPARK-31312]: 
Transforming Hive simple UDF (using JAR) expression may incur CNFE in later 
evaluation
+  https://issues.apache.org/jira/browse/SPARK-31420;>[SPARK-31420]: 
Infinite timeline redraw in job details page
+  https://issues.apache.org/jira/browse/SPARK-31485;>[SPARK-31485]: 
Barrier stage can hang if only partial tasks launched
+  https://issues.apache.org/jira/browse/SPARK-31500;>[SPARK-31500]: 
collect_set() of BinaryType returns duplicate elements
+  https://issues.apache.org/jira/browse/SPARK-31503;>[SPARK-31503]: fix 
the SQL string of the TRIM functions
+  https://issues.apache.org/jira/browse/SPARK-31663;>[SPARK-31663]: 
Grouping sets with having clause returns the wrong result
+  https://issues.apache.org/jira/browse/SPARK-26908;>[SPARK-26908]: Fix 
toMilis
+  https://issues.apache.org/jira/browse/SPARK-31563;>[SPARK-31563]: 
Failure of Inset.sql for UTF8String collection
+
+
+Dependency Changes
+
+While being a maintence release we did still upgrade some dependencies in 
this release they are:
+
+  netty-all to 4.1.47.Final (https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2019-20445;>[CVE-2019-20445])
+  Janino to 3.0.16 (SQL Generated code)
+  aws-java-sdk-sts to 1.11.655 (required for kinesis client upgrade)
+  snappy 1.1.7.5 (stability improvements  ppc64le performance)
+
+
+Known issues
+
+  htt

[spark] branch branch-3.0 updated: [SPARK-31934][BUILD] Remove set -x from docker image tool

2020-06-08 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new bd0f5f2  [SPARK-31934][BUILD] Remove set -x from docker image tool
bd0f5f2 is described below

commit bd0f5f2c57ae954d29b0940c1cff77cd2fff8ed7
Author: Holden Karau 
AuthorDate: Mon Jun 8 16:03:13 2020 -0700

[SPARK-31934][BUILD] Remove set -x from docker image tool

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

Remove `set -x` from the docker image tool.

### Why are the changes needed?

The image tool prints out information that may confusing.

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

Less information is displayed by the docker image tool.

### How was this patch tested?

Ran docker image tool locally.

Closes #28759 from 
holdenk/SPARK-31934-remove-extranious-info-from-the-docker-image-tool.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
(cherry picked from commit 06959ebc399e4fa6a90c30e4f0c897cad1f6a496)
Signed-off-by: Holden Karau 
---
 bin/docker-image-tool.sh | 2 --
 1 file changed, 2 deletions(-)

diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh
index 8a01b80..6d74f83 100755
--- a/bin/docker-image-tool.sh
+++ b/bin/docker-image-tool.sh
@@ -19,8 +19,6 @@
 # This script builds and pushes docker images when run from a release of Spark
 # with Kubernetes support.
 
-set -x
-
 function error {
   echo "$@" 1>&2
   exit 1


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



[spark] branch master updated: [SPARK-31934][BUILD] Remove set -x from docker image tool

2020-06-08 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 06959eb  [SPARK-31934][BUILD] Remove set -x from docker image tool
06959eb is described below

commit 06959ebc399e4fa6a90c30e4f0c897cad1f6a496
Author: Holden Karau 
AuthorDate: Mon Jun 8 16:03:13 2020 -0700

[SPARK-31934][BUILD] Remove set -x from docker image tool

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

Remove `set -x` from the docker image tool.

### Why are the changes needed?

The image tool prints out information that may confusing.

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

Less information is displayed by the docker image tool.

### How was this patch tested?

Ran docker image tool locally.

Closes #28759 from 
holdenk/SPARK-31934-remove-extranious-info-from-the-docker-image-tool.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 bin/docker-image-tool.sh | 2 --
 1 file changed, 2 deletions(-)

diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh
index 8a01b80..6d74f83 100755
--- a/bin/docker-image-tool.sh
+++ b/bin/docker-image-tool.sh
@@ -19,8 +19,6 @@
 # This script builds and pushes docker images when run from a release of Spark
 # with Kubernetes support.
 
-set -x
-
 function error {
   echo "$@" 1>&2
   exit 1


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



svn commit: r39955 - /dev/spark/v2.4.6-rc8-docs/

2020-06-05 Thread holden
Author: holden
Date: Sat Jun  6 00:09:52 2020
New Revision: 39955

Log:
Remove RC artifacts

Removed:
dev/spark/v2.4.6-rc8-docs/


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



[spark] tag v2.4.6 created (now 807e0a4)

2020-06-05 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a change to tag v2.4.6
in repository https://gitbox.apache.org/repos/asf/spark.git.


  at 807e0a4  (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: r39950 - /release/spark/KEYS

2020-06-05 Thread holden
Author: holden
Date: Fri Jun  5 18:03:18 2020
New Revision: 39950

Log:
Update KEYS

Modified:
release/spark/KEYS

Modified: release/spark/KEYS
==
--- release/spark/KEYS (original)
+++ release/spark/KEYS Fri Jun  5 18:03:18 2020
@@ -1167,3 +1167,188 @@ rMA+YcuC9o2K7dKjVv3KinQ2Tiv4TVxyTjcyZurg
 0TbepIdiQlc=
 =wdlY
 -END PGP PUBLIC KEY BLOCK-
+
+pub   rsa4096 2020-03-30 [SC]
+  4A8BDA48E6E212A734632502DEA963E2E9347D66
+uid   [ultimate] Reynold Xin (CODE SIGNING KEY) 
+sub   rsa4096 2020-03-30 [E]
+
+-BEGIN PGP PUBLIC KEY BLOCK-
+
+mQINBF6BkJkBEACmRKcV6c575E6jOyZBwLteV7hJsETNYx9jMkENiyeyTFJ3A8Hg
++gPAmoU6jvzugR98qgVSH0uj/HZH1zEkJx049+OHwBcZ48mGJakIaKcg3k1CPRTL
+VDRWg7M4P7nQisMHsPHrdGPJFVBE7Mn6pafuRZ46gtnXf2Ec1EsvMBOYjRNt6nSg
+GvoQdiv5SjUuwxfrw7CICj1agxwLarBcWpIF6PMU7yG+XjTIrSM63KuuV+fOZvKM
+AdjwwUNNj2aOkprPHfmFIgSnEMsxvoJQNqYTaWzwT8WAyW1qTd0LhYYDTnb4J+j2
+BxgG5ASHYpsLQ1Moy+lYsTxWsoZMvqTqv/h+Mlb8fiUTiYppeMnLzxtI/t8Trvt8
+rXNGSkNd8dM5uqJ9Ba2MS6UB6EZUd5e7aPy8z5ThlhygRjLk0527O4BYAWlZw5F8
+egq/X0liCeRHoFUsyNnuQYSqo2spdTIV2ExKo/hEF1FgbXF6s1v/TcfzS0PkSYEH
+5yhKYoEkYOXIneIjUasy8xM9O2578NsVu1GH0n+E29KDA0w+QKwpbjgb9VWKCjk1
+CPvK7oi3DKA4A28w/h5jI9Xzb343L0gb+IhdgL5lNWp2HoSy+y7Smnbz6IchjAP7
+zCtQ9ZJCLdXgCtDlXUeF+TXzEfKUYwa0jnha/fArM3PVGvQlWdpVhe/oLQARAQAB
+tDBSZXlub2xkIFhpbiAoQ09ERSBTSUdOSU5HIEtFWSkgPHJ4aW5AYXBhY2hlLm9y
+Zz6JAk4EEwEIADgWIQRKi9pI5uISpzRjJQLeqWPi6TR9ZgUCXoGQmQIbAwULCQgH
+AgYVCgkICwIEFgIDAQIeAQIXgAAKCRDeqWPi6TR9ZrBJEACW92VdruNL+dYYH0Cu
+9oxZx0thCE1twc/6rvgvIj//0kZ4ZA6RoDId8vSmKSkB0GwMT7daIoeIvRTiEdMQ
+Wai7zqvNEdT1qdNn7MfN1rveN1tBNVndzbZ8S8Nz4sqZ/8R3wG90c2XLwno3joXA
+FhFRfVa+TWI1Ux84/ZXuzD14f54dorVo0CT51CnU67ERBAijl7UugPM3Fs7ApU/o
+SWCMq7ScPde81jmgMqBDLcj/hueCOTU5m8irOGGY439qEF+H41I+IB60yzAS4Gez
+xZl55Mv7ZKdwWtCcwtUYIm4R8NNu4alTxUpxw4ttRW3Kzue78TOIMTWTwRKrP5t2
+yq9bMT1fSO7h/Ntn8dXUL0EM/h+6k5py5Kr0+mrV/s0Z530Fit6AC/ReWV6hSGdk
+F1Z1ECa4AoUHqtoQKL+CNgO2qlJn/sKj3g10NiSwqUdUuxCSOpsY72udRLG9tfkB
+OwW3lTKLp66gYYE3nYaHzJKGdRs7aJ8RRALMQkadsyqpdVMp+Yvbj/3Hn3uB3jTt
+S+RolH545toeuhXaiIWlm2434oHW6QjzpPwaNp5AiWm+vMfPkhhCX6WT0jv9nEtM
+kJJVgwlWNKYEW9nLaIRMWWONSy9aJapZfLW0XDiKidibPHqNFih9z49eDVLobi5e
+mzmOFkKFxs9D4sg9oVmId6Y9SbkCDQRegZCZARAA5ZMv1ki5mKJVpASRGfTHVH5o
+9HixwJOinkHjSK3zFpuvh0bs+rKZL2+TUXci9Em64xXuYbiGH3YgH061H9tgAMaN
+iSIFGPlbBPbduJjdiUALqauOjjCIoWJLyuAC25zSGCeAwzQiRXN6VJUYwjQnDMDG
+8iUyL+IdXjq2T6vFVZGR/uVteRqqvEcg9km6IrFmXefqfry4hZ5a7SbmThCHqGxx
+5Oy+VkWw1IP7fHIUdC9ie45X6n08yC2BfWI4+RBny8906pSXEN/ag0Yw7vWkiyuK
+wZsoe0pRczV8mx6QF2+oJjRMtziKYW72jKE9a/DXXzQ3Luq5gyZeq0cluYNGHVdj
+ijA2ORNLloAfGjVGRKVznUFN8LMkcxm4jiiHKRkZEcjgm+1tRzGPufFidyhQIYO2
+YCOpnPQh5IXznb3RZ0JqJcXdne+7Nge85URTEMmMyx5kXvD03ZmUObshDL12YoM3
+bGzObo6jYg+h38Xlx9+9QAwGkf+gApIPI8KqPAVyP6s60AR4iR6iehEOciz7h6/b
+T9bKMw0w9cvyJzY1IJsy2sQYFwNyHYWQkyDciRAmIwriHhBDfXdBodF95V3uGbIp
+DZw3jVxcgJWKZ3y65N1aCguEI1fyy9JU12++GMBa+wuv9kdhSoj2qgInFB1VXGC7
+bBlRnHB44tsFTBEqqOcAEQEAAYkCNgQYAQgAIBYhBEqL2kjm4hKnNGMlAt6pY+Lp
+NH1mBQJegZCZAhsMAAoJEN6pY+LpNH1mwIYQAIRqbhEjL6uMxM19OMPDydbhiWoI
+8BmoqzsvRNF9VidjPRicYJ5JL5FFvvTyT6g87L8aRhiAdX/la92PdJ9DTS3sfIKF
+pIcUDFybKgk4pmGWl0fNIwEjHewf6HlndCFmVuPe32V/ZkCwb58dro15xzxblckB
+kgsqb0Xbfz/3Iwlqr5eTKH5iPrDFcYKy1ODcFmXS+udMm5uwn+d/RNmj8B3kgwrw
+brs53264qdWbfsxGPC1ZkDNNSRyIy6wGvc/diRm4TSV/Lmd5OoDX4UkPJ++JhGoO
+cYKxc2KzrEZxzMgJ3xFRs3zeymOwtgXUU1GBCuD7uxr1vacFwUV+9ymTeyUdTxB3
++/DzxYOJGQL/3IXlyQ2azoCWUpCjW0MFM1OolragOFJeQ+V0xrlOiXXAFfHo0KPG
+y0QdK810Ok+XYR6U9Y7yb6tYDgi+w9r46XjurdiZnUxxLUpFG++tSgBQ5X4y2UGw
+C4n0T8/jn6KIUZ0kx51ZZ6CEChjBt+AU+HCnw2sZfgq8Nlos95tw2MT6kn8BrY68
+n297ev/1T6B0OasQaw3Itw29+T+FdzdU4c6XW/rC6VAlBikWIS5zCT//vAeBacxL
+HYoqwKL52HzG121lfWXhx5vNF4bg/fKrFEOy2Wp1fMG6nRcuUUROvieD6ZU4ZrLA
+NjpTIP+lOkfxRwUi
+=rggH
+-END PGP PUBLIC KEY BLOCK-
+
+pub   rsa4096 2018-07-01 [SC]
+  CA19E2CC201B6522D3A2A05DC41AB674424FC3FB
+uid   [ultimate] Holden Karau (CODE SIGNING KEY) 
+sub   rsa4096 2018-07-01 [E]
+
+-BEGIN PGP PUBLIC KEY BLOCK-
+
+mQINBFs5MrIBEAC1jGSgkZd+HjtAt5Mlh9LKOucaHRN9ItLxk4rrYklxt3YI20ft
+DIrLiAQCotiYYhmhBaXZRTfgGgusRa6MhdmX+5t/+yKZfoLqfeZeKawDsvKhGgod
+Yl5iCxB0OmwdZjjOaZPDW2zVlJkjuB24SnSNwsBnd0kkDAEoGMN3bUflvnzL2EJI
+zzvmR0HGK0aWRFzj7fvBu5sN4RM03ZN/3CCCVHlznH8ILvFDz8PnLf+nMhBFIQOo
+OBiiLq20Ag19vj1rXjfe3U0YT6L4+SbLAt6vde8YpXs/mpkWvVPH4OtEATKsLqvJ
+5pP8YGxOkPfls6lfziyLV/JMUO6f9BxqgXXWLPvjAuhRWKJ+yyFfWa4Ju8PTKEO6
+Z344tl+1FNNXNkBGTVN+k264lKe/He8ywYZKgtyF9iGvOf8HgUMsjg6DSFOMaZ9I
+oFfe1G1zYtDCwV4rkf+AS/SlxaMcQpr96wxhtnBL6zD92gK8Of1+EN4yrfVSp8Gj
+cnRC3jYC4iUtyWDZcUpHkSajxH2IbYQMty8fHhYLA6mq8HEhPEtsSE3eX1KQEZEX
+O+mcjHaC6XuvC7aQozNkYw19S6vK9l7Dwcq7+0WYpHFkQ5RgTmE7ns36/lq/4s/B
+9TTC9m8eXAjBtjNjlnax7qXBshru6dYt9kfI4PM6SV0WHZtSHldYkqy/GwARAQAB
+tDNIb2xkZW4gS2FyYXUgKENPREUgU0lHTklORyBLRVkpIDxob2xkZW5AYXBhY2hl
+Lm9yZz6JAk4EEwEKADgWIQTKGeLMIBtlItOioF3EGrZ0Qk/D+wUCWzkysgIbAwUL
+CQgHAgYVCgkICwIEFgIDAQIeAQIXgAAKCRDEGrZ0Qk/D+4pwEACUTTtP2xxp7oE/
+1V7ozsvrhs0TwaqVVpBnhgvtj8iW8towVRPiBw+f

svn commit: r39949 - /dev/spark/v2.4.6-rc8-bin/ /release/spark/spark-2.4.6/

2020-06-05 Thread holden
Author: holden
Date: Fri Jun  5 18:01:51 2020
New Revision: 39949

Log:
Release Spark 2.4.6 RC8 :)

Added:
release/spark/spark-2.4.6/
  - copied from r39948, dev/spark/v2.4.6-rc8-bin/
Removed:
dev/spark/v2.4.6-rc8-bin/


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



[spark] branch branch-3.0 updated: [SPARK-31778][K8S][BUILD] Support cross-building docker images

2020-06-02 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new 1292bf2  [SPARK-31778][K8S][BUILD] Support cross-building docker images
1292bf2 is described below

commit 1292bf22bf0baa60c902a8c6985bc508cbdbef1b
Author: Holden Karau 
AuthorDate: Tue Jun 2 11:11:23 2020 -0700

[SPARK-31778][K8S][BUILD] Support cross-building docker images

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

Add cross build support to our docker image script using the new dockerx 
extension.

### Why are the changes needed?

We have a CI for Spark on ARM, we should support building images for ARM 
and AMD64.

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

Yes, a new flag is added to the docker image build script to cross-build

### How was this patch tested?
Manually ran build script & pushed to 
https://hub.docker.com/repository/registry-1.docker.io/holdenk/spark/tags?page=1
 verified amd64 & arm64 listed.

Closes #28615 from holdenk/cross-build.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
(cherry picked from commit 25702281dc0c7cc333978f51a15ebf9fd02cc684)
Signed-off-by: Holden Karau 
---
 bin/docker-image-tool.sh | 30 +-
 1 file changed, 29 insertions(+), 1 deletion(-)

diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh
index 57b8625..8a01b80 100755
--- a/bin/docker-image-tool.sh
+++ b/bin/docker-image-tool.sh
@@ -19,6 +19,8 @@
 # This script builds and pushes docker images when run from a release of Spark
 # with Kubernetes support.
 
+set -x
+
 function error {
   echo "$@" 1>&2
   exit 1
@@ -172,6 +174,7 @@ function build {
   local 
BASEDOCKERFILE=${BASEDOCKERFILE:-"kubernetes/dockerfiles/spark/Dockerfile"}
   local PYDOCKERFILE=${PYDOCKERFILE:-false}
   local RDOCKERFILE=${RDOCKERFILE:-false}
+  local ARCHS=${ARCHS:-"--platform linux/amd64,linux/arm64"}
 
   (cd $(img_ctx_dir base) && docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
 -t $(image_ref spark) \
@@ -179,6 +182,11 @@ function build {
   if [ $? -ne 0 ]; then
 error "Failed to build Spark JVM Docker image, please refer to Docker 
build output for details."
   fi
+  if [ "${CROSS_BUILD}" != "false" ]; then
+  (cd $(img_ctx_dir base) && docker buildx build $ARCHS $NOCACHEARG 
"${BUILD_ARGS[@]}" \
+-t $(image_ref spark) \
+-f "$BASEDOCKERFILE" .)
+  fi
 
   if [ "${PYDOCKERFILE}" != "false" ]; then
 (cd $(img_ctx_dir pyspark) && docker build $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
@@ -187,6 +195,11 @@ function build {
   if [ $? -ne 0 ]; then
 error "Failed to build PySpark Docker image, please refer to Docker 
build output for details."
   fi
+  if [ "${CROSS_BUILD}" != "false" ]; then
+(cd $(img_ctx_dir pyspark) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
+  -t $(image_ref spark-py) \
+  -f "$PYDOCKERFILE" .)
+  fi
   fi
 
   if [ "${RDOCKERFILE}" != "false" ]; then
@@ -196,6 +209,11 @@ function build {
 if [ $? -ne 0 ]; then
   error "Failed to build SparkR Docker image, please refer to Docker build 
output for details."
 fi
+if [ "${CROSS_BUILD}" != "false" ]; then
+  (cd $(img_ctx_dir sparkr) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
+-t $(image_ref spark-r) \
+-f "$RDOCKERFILE" .)
+fi
   fi
 }
 
@@ -227,6 +245,8 @@ Options:
   -nBuild docker image with --no-cache
   -u uidUID to use in the USER directive to set the user the 
main Spark process runs as inside the
 resulting container
+  -XUse docker buildx to cross build. Automatically pushes.
+See 
https://docs.docker.com/buildx/working-with-buildx/ for steps to setup buildx.
   -b argBuild arg to build or push the image. For multiple 
build args, this option needs to
 be used separately for each build arg.
 
@@ -252,6 +272,12 @@ Examples:
   - Build and push JDK11-based image with tag "v3.0.0" to docker.io/myrepo
 $0 -r docker.io/myrepo -t v3.0.0 -b java_image_tag=11-jre-slim build
 $0 -r docker.io/myrepo -t v3.0.0 push
+
+  - Build and push JDK11-based image for multiple archs to docker.io/myrepo
+$0 -r docker.io/myrepo -t v3.0.0 -X -b java_image_tag=11-jre-slim build
+# Note: buildx, which does 

[spark] branch master updated: [SPARK-31778][K8S][BUILD] Support cross-building docker images

2020-06-02 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 2570228  [SPARK-31778][K8S][BUILD] Support cross-building docker images
2570228 is described below

commit 25702281dc0c7cc333978f51a15ebf9fd02cc684
Author: Holden Karau 
AuthorDate: Tue Jun 2 11:11:23 2020 -0700

[SPARK-31778][K8S][BUILD] Support cross-building docker images

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

Add cross build support to our docker image script using the new dockerx 
extension.

### Why are the changes needed?

We have a CI for Spark on ARM, we should support building images for ARM 
and AMD64.

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

Yes, a new flag is added to the docker image build script to cross-build

### How was this patch tested?
Manually ran build script & pushed to 
https://hub.docker.com/repository/registry-1.docker.io/holdenk/spark/tags?page=1
 verified amd64 & arm64 listed.

Closes #28615 from holdenk/cross-build.

Lead-authored-by: Holden Karau 
Co-authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 bin/docker-image-tool.sh | 30 +-
 1 file changed, 29 insertions(+), 1 deletion(-)

diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh
index 57b8625..8a01b80 100755
--- a/bin/docker-image-tool.sh
+++ b/bin/docker-image-tool.sh
@@ -19,6 +19,8 @@
 # This script builds and pushes docker images when run from a release of Spark
 # with Kubernetes support.
 
+set -x
+
 function error {
   echo "$@" 1>&2
   exit 1
@@ -172,6 +174,7 @@ function build {
   local 
BASEDOCKERFILE=${BASEDOCKERFILE:-"kubernetes/dockerfiles/spark/Dockerfile"}
   local PYDOCKERFILE=${PYDOCKERFILE:-false}
   local RDOCKERFILE=${RDOCKERFILE:-false}
+  local ARCHS=${ARCHS:-"--platform linux/amd64,linux/arm64"}
 
   (cd $(img_ctx_dir base) && docker build $NOCACHEARG "${BUILD_ARGS[@]}" \
 -t $(image_ref spark) \
@@ -179,6 +182,11 @@ function build {
   if [ $? -ne 0 ]; then
 error "Failed to build Spark JVM Docker image, please refer to Docker 
build output for details."
   fi
+  if [ "${CROSS_BUILD}" != "false" ]; then
+  (cd $(img_ctx_dir base) && docker buildx build $ARCHS $NOCACHEARG 
"${BUILD_ARGS[@]}" \
+-t $(image_ref spark) \
+-f "$BASEDOCKERFILE" .)
+  fi
 
   if [ "${PYDOCKERFILE}" != "false" ]; then
 (cd $(img_ctx_dir pyspark) && docker build $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
@@ -187,6 +195,11 @@ function build {
   if [ $? -ne 0 ]; then
 error "Failed to build PySpark Docker image, please refer to Docker 
build output for details."
   fi
+  if [ "${CROSS_BUILD}" != "false" ]; then
+(cd $(img_ctx_dir pyspark) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
+  -t $(image_ref spark-py) \
+  -f "$PYDOCKERFILE" .)
+  fi
   fi
 
   if [ "${RDOCKERFILE}" != "false" ]; then
@@ -196,6 +209,11 @@ function build {
 if [ $? -ne 0 ]; then
   error "Failed to build SparkR Docker image, please refer to Docker build 
output for details."
 fi
+if [ "${CROSS_BUILD}" != "false" ]; then
+  (cd $(img_ctx_dir sparkr) && docker buildx build $ARCHS $NOCACHEARG 
"${BINDING_BUILD_ARGS[@]}" \
+-t $(image_ref spark-r) \
+-f "$RDOCKERFILE" .)
+fi
   fi
 }
 
@@ -227,6 +245,8 @@ Options:
   -nBuild docker image with --no-cache
   -u uidUID to use in the USER directive to set the user the 
main Spark process runs as inside the
 resulting container
+  -XUse docker buildx to cross build. Automatically pushes.
+See 
https://docs.docker.com/buildx/working-with-buildx/ for steps to setup buildx.
   -b argBuild arg to build or push the image. For multiple 
build args, this option needs to
 be used separately for each build arg.
 
@@ -252,6 +272,12 @@ Examples:
   - Build and push JDK11-based image with tag "v3.0.0" to docker.io/myrepo
 $0 -r docker.io/myrepo -t v3.0.0 -b java_image_tag=11-jre-slim build
 $0 -r docker.io/myrepo -t v3.0.0 push
+
+  - Build and push JDK11-based image for multiple archs to docker.io/myrepo
+$0 -r docker.io/myrepo -t v3.0.0 -X -b java_image_tag=11-jre-slim build
+# Note: buildx, which does cross building, needs to do the push during 
build
+# So there is no seperate push step with -X
+
 EOF
 }
 
@@ 

[spark] branch branch-3.0 updated: [SPARK-31860][BUILD] only push release tags on succes

2020-06-02 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new b814cbc  [SPARK-31860][BUILD] only push release tags on succes
b814cbc is described below

commit b814cbc8121c8d02a81efbae3e3cf1b6a6219262
Author: Holden Karau 
AuthorDate: Tue Jun 2 11:04:07 2020 -0700

[SPARK-31860][BUILD] only push release tags on succes

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

Only push the release tag after the build has finished.

### Why are the changes needed?

If the build fails we don't need a release tag.

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

### How was this patch tested?
Running locally with a fake user in 
https://github.com/apache/spark/pull/28667

Closes #28700 from 
holdenk/SPARK-31860-build-master-only-push-tags-on-success.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
(cherry picked from commit 69ba9b662e2ace592380e3cc5de041031dec2254)
Signed-off-by: Holden Karau 
---
 dev/create-release/do-release.sh| 14 +++---
 dev/create-release/release-build.sh |  7 +--
 dev/create-release/release-tag.sh   | 16 +++-
 3 files changed, 19 insertions(+), 18 deletions(-)

diff --git a/dev/create-release/do-release.sh b/dev/create-release/do-release.sh
index 4f18a55..64fba8a 100755
--- a/dev/create-release/do-release.sh
+++ b/dev/create-release/do-release.sh
@@ -17,6 +17,8 @@
 # limitations under the License.
 #
 
+set -e
+
 SELF=$(cd $(dirname $0) && pwd)
 . "$SELF/release-util.sh"
 
@@ -52,9 +54,6 @@ function should_build {
 if should_build "tag" && [ $SKIP_TAG = 0 ]; then
   run_silent "Creating release tag $RELEASE_TAG..." "tag.log" \
 "$SELF/release-tag.sh"
-  echo "It may take some time for the tag to be synchronized to github."
-  echo "Press enter when you've verified that the new tag ($RELEASE_TAG) is 
available."
-  read
 else
   echo "Skipping tag creation for $RELEASE_TAG."
 fi
@@ -79,3 +78,12 @@ if should_build "publish"; then
 else
   echo "Skipping publish step."
 fi
+
+if should_build "tag" && [ $SKIP_TAG = 0 ]; then
+  git push origin $RELEASE_TAG
+  if [[ $RELEASE_TAG != *"preview"* ]]; then
+git push origin HEAD:$GIT_BRANCH
+  else
+echo "It's preview release. We only push $RELEASE_TAG to remote."
+  fi
+fi
diff --git a/dev/create-release/release-build.sh 
b/dev/create-release/release-build.sh
index e3bcb72..66c5184 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -92,9 +92,12 @@ BASE_DIR=$(pwd)
 init_java
 init_maven_sbt
 
-rm -rf spark
-git clone "$ASF_REPO"
+# Only clone repo fresh if not present, otherwise use checkout from the tag 
step
+if [ ! -d spark ]; then
+  git clone "$ASF_REPO"
+fi
 cd spark
+git fetch
 git checkout $GIT_REF
 git_hash=`git rev-parse --short HEAD`
 echo "Checked out Spark git hash $git_hash"
diff --git a/dev/create-release/release-tag.sh 
b/dev/create-release/release-tag.sh
index 39856a9..e37aa27 100755
--- a/dev/create-release/release-tag.sh
+++ b/dev/create-release/release-tag.sh
@@ -25,6 +25,7 @@ function exit_with_usage {
   cat << EOF
 usage: $NAME
 Tags a Spark release on a particular branch.
+You must push the tags after.
 
 Inputs are specified with the following environment variables:
 ASF_USERNAME - Apache Username
@@ -105,19 +106,8 @@ sed -i".tmp7" 
's/SPARK_VERSION_SHORT:.*$/SPARK_VERSION_SHORT: '"$R_NEXT_VERSION"
 
 git commit -a -m "Preparing development version $NEXT_VERSION"
 
-if ! is_dry_run; then
-  # Push changes
-  git push origin $RELEASE_TAG
-  if [[ $RELEASE_VERSION != *"preview"* ]]; then
-git push origin HEAD:$GIT_BRANCH
-  else
-echo "It's preview release. We only push $RELEASE_TAG to remote."
-  fi
-
-  cd ..
-  rm -rf spark
-else
-  cd ..
+cd ..
+if is_dry_run; then
   mv spark spark.tag
   echo "Clone with version changes and tag available as spark.tag in the 
output directory."
 fi


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



[spark] branch master updated: [SPARK-31860][BUILD] only push release tags on succes

2020-06-02 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 69ba9b6  [SPARK-31860][BUILD] only push release tags on succes
69ba9b6 is described below

commit 69ba9b662e2ace592380e3cc5de041031dec2254
Author: Holden Karau 
AuthorDate: Tue Jun 2 11:04:07 2020 -0700

[SPARK-31860][BUILD] only push release tags on succes

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

Only push the release tag after the build has finished.

### Why are the changes needed?

If the build fails we don't need a release tag.

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

### How was this patch tested?
Running locally with a fake user in 
https://github.com/apache/spark/pull/28667

Closes #28700 from 
holdenk/SPARK-31860-build-master-only-push-tags-on-success.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 dev/create-release/do-release.sh| 14 +++---
 dev/create-release/release-build.sh |  7 +--
 dev/create-release/release-tag.sh   | 16 +++-
 3 files changed, 19 insertions(+), 18 deletions(-)

diff --git a/dev/create-release/do-release.sh b/dev/create-release/do-release.sh
index 4f18a55..64fba8a 100755
--- a/dev/create-release/do-release.sh
+++ b/dev/create-release/do-release.sh
@@ -17,6 +17,8 @@
 # limitations under the License.
 #
 
+set -e
+
 SELF=$(cd $(dirname $0) && pwd)
 . "$SELF/release-util.sh"
 
@@ -52,9 +54,6 @@ function should_build {
 if should_build "tag" && [ $SKIP_TAG = 0 ]; then
   run_silent "Creating release tag $RELEASE_TAG..." "tag.log" \
 "$SELF/release-tag.sh"
-  echo "It may take some time for the tag to be synchronized to github."
-  echo "Press enter when you've verified that the new tag ($RELEASE_TAG) is 
available."
-  read
 else
   echo "Skipping tag creation for $RELEASE_TAG."
 fi
@@ -79,3 +78,12 @@ if should_build "publish"; then
 else
   echo "Skipping publish step."
 fi
+
+if should_build "tag" && [ $SKIP_TAG = 0 ]; then
+  git push origin $RELEASE_TAG
+  if [[ $RELEASE_TAG != *"preview"* ]]; then
+git push origin HEAD:$GIT_BRANCH
+  else
+echo "It's preview release. We only push $RELEASE_TAG to remote."
+  fi
+fi
diff --git a/dev/create-release/release-build.sh 
b/dev/create-release/release-build.sh
index e3bcb72..66c5184 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -92,9 +92,12 @@ BASE_DIR=$(pwd)
 init_java
 init_maven_sbt
 
-rm -rf spark
-git clone "$ASF_REPO"
+# Only clone repo fresh if not present, otherwise use checkout from the tag 
step
+if [ ! -d spark ]; then
+  git clone "$ASF_REPO"
+fi
 cd spark
+git fetch
 git checkout $GIT_REF
 git_hash=`git rev-parse --short HEAD`
 echo "Checked out Spark git hash $git_hash"
diff --git a/dev/create-release/release-tag.sh 
b/dev/create-release/release-tag.sh
index 39856a9..e37aa27 100755
--- a/dev/create-release/release-tag.sh
+++ b/dev/create-release/release-tag.sh
@@ -25,6 +25,7 @@ function exit_with_usage {
   cat << EOF
 usage: $NAME
 Tags a Spark release on a particular branch.
+You must push the tags after.
 
 Inputs are specified with the following environment variables:
 ASF_USERNAME - Apache Username
@@ -105,19 +106,8 @@ sed -i".tmp7" 
's/SPARK_VERSION_SHORT:.*$/SPARK_VERSION_SHORT: '"$R_NEXT_VERSION"
 
 git commit -a -m "Preparing development version $NEXT_VERSION"
 
-if ! is_dry_run; then
-  # Push changes
-  git push origin $RELEASE_TAG
-  if [[ $RELEASE_VERSION != *"preview"* ]]; then
-git push origin HEAD:$GIT_BRANCH
-  else
-echo "It's preview release. We only push $RELEASE_TAG to remote."
-  fi
-
-  cd ..
-  rm -rf spark
-else
-  cd ..
+cd ..
+if is_dry_run; then
   mv spark spark.tag
   echo "Clone with version changes and tag available as spark.tag in the 
output directory."
 fi


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



[spark] branch branch-2.4 updated: [SPARK-31860][BUILD][2.4] only push release tags on success

2020-06-01 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new bf01280  [SPARK-31860][BUILD][2.4] only push release tags on success
bf01280 is described below

commit bf012809eb28a7feb5f5e20788550a39e1dbe8ad
Author: Holden Karau 
AuthorDate: Mon Jun 1 17:46:13 2020 -0700

[SPARK-31860][BUILD][2.4] only push release tags on success

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

Only push the release tag after the build has finished.

### Why are the changes needed?

If the build fails we don't need a release tag.

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

### How was this patch tested?
Running locally with a fake user

Closes #28667 from holdenk/SPARK-31860-only-push-release-tags-on-success.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 dev/create-release/do-release.sh| 11 ---
 dev/create-release/release-build.sh |  7 +--
 dev/create-release/release-tag.sh   | 11 ++-
 3 files changed, 15 insertions(+), 14 deletions(-)

diff --git a/dev/create-release/do-release.sh b/dev/create-release/do-release.sh
index f1d4f3a..9ff 100755
--- a/dev/create-release/do-release.sh
+++ b/dev/create-release/do-release.sh
@@ -17,6 +17,8 @@
 # limitations under the License.
 #
 
+set -e
+
 SELF=$(cd $(dirname $0) && pwd)
 . "$SELF/release-util.sh"
 
@@ -52,9 +54,6 @@ function should_build {
 if should_build "tag" && [ $SKIP_TAG = 0 ]; then
   run_silent "Creating release tag $RELEASE_TAG..." "tag.log" \
 "$SELF/release-tag.sh"
-  echo "It may take some time for the tag to be synchronized to github."
-  echo "Press enter when you've verified that the new tag ($RELEASE_TAG) is 
available."
-  read
 else
   echo "Skipping tag creation for $RELEASE_TAG."
 fi
@@ -79,3 +78,9 @@ if should_build "publish"; then
 else
   echo "Skipping publish step."
 fi
+
+if should_build "tag" && [ $SKIP_TAG = 0 ]; then
+  # Push the tag after success
+  git push origin "$RELEASE_TAG"
+  git push origin "HEAD:$GIT_BRANCH"
+fi
diff --git a/dev/create-release/release-build.sh 
b/dev/create-release/release-build.sh
index 1fd8a30..3c287fd 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -92,9 +92,12 @@ BASE_DIR=$(pwd)
 init_java
 init_maven_sbt
 
-rm -rf spark
-git clone "$ASF_REPO"
+# Only clone the repo fresh when not present, otherwise use checkout
+if [ ! -d spark ]; then
+  git clone "$ASF_REPO"
+fi
 cd spark
+git fetch
 git checkout $GIT_REF
 git_hash=`git rev-parse --short HEAD`
 echo "Checked out Spark git hash $git_hash"
diff --git a/dev/create-release/release-tag.sh 
b/dev/create-release/release-tag.sh
index 8024440..517c7f7 100755
--- a/dev/create-release/release-tag.sh
+++ b/dev/create-release/release-tag.sh
@@ -24,7 +24,7 @@ function exit_with_usage {
   local NAME=$(basename $0)
   cat << EOF
 usage: $NAME
-Tags a Spark release on a particular branch.
+Tags a Spark release on a particular branch. Must push after
 
 Inputs are specified with the following environment variables:
 ASF_USERNAME - Apache Username
@@ -101,14 +101,7 @@ sed -i".tmp7" 
's/SPARK_VERSION_SHORT:.*$/SPARK_VERSION_SHORT: '"$R_NEXT_VERSION"
 
 git commit -a -m "Preparing development version $NEXT_VERSION"
 
-if ! is_dry_run; then
-  # Push changes
-  git push origin $RELEASE_TAG
-  git push origin HEAD:$GIT_BRANCH
-
-  cd ..
-  rm -rf spark
-else
+if is_dry_run; then
   cd ..
   mv spark spark.tag
   echo "Clone with version changes and tag available as spark.tag in the 
output directory."


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



[spark] branch branch-2.4 updated: [SPARK-31889][BUILD] Docker release script does not allocate enough memory to reliably publish

2020-06-01 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new ac59a8c  [SPARK-31889][BUILD] Docker release script does not allocate 
enough memory to reliably publish
ac59a8c is described below

commit ac59a8ccee0f3121fd87477b99223c6d3690a936
Author: Holden Karau 
AuthorDate: Mon Jun 1 15:49:17 2020 -0700

[SPARK-31889][BUILD] Docker release script does not allocate enough memory 
to reliably publish

### What changes were proposed in this pull request?
Allow overriding the zinc options in the docker release and set a higher so 
the publish step can succeed consistently.

### Why are the changes needed?

The publish step experiences memory pressure.

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

No

### How was this patch tested?
Running test locally with fake user to see if publish step (besides svn 
part) succeeds

Closes #28698 from 
holdenk/SPARK-31889-docker-release-script-does-not-allocate-enough-memory-to-reliably-publish.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
(cherry picked from commit ab9e5a2fe9a0e9e93301096a027d7409fc3c9b64)
Signed-off-by: Holden Karau 
---
 dev/create-release/do-release-docker.sh | 1 +
 1 file changed, 1 insertion(+)

diff --git a/dev/create-release/do-release-docker.sh 
b/dev/create-release/do-release-docker.sh
index f643c06..8ea75b8 100755
--- a/dev/create-release/do-release-docker.sh
+++ b/dev/create-release/do-release-docker.sh
@@ -128,6 +128,7 @@ ASF_PASSWORD=$ASF_PASSWORD
 GPG_PASSPHRASE=$GPG_PASSPHRASE
 RELEASE_STEP=$RELEASE_STEP
 USER=$USER
+ZINC_OPTS=${RELEASE_ZINC_OPTS:-"-Xmx4g -XX:ReservedCodeCacheSize=2g"}
 EOF
 
 JAVA_VOL=


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



[spark] branch branch-3.0 updated: [SPARK-31889][BUILD] Docker release script does not allocate enough memory to reliably publish

2020-06-01 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.0 by this push:
 new 387f57c  [SPARK-31889][BUILD] Docker release script does not allocate 
enough memory to reliably publish
387f57c is described below

commit 387f57c574beb726db4d305f3c1da7d8255ad103
Author: Holden Karau 
AuthorDate: Mon Jun 1 15:49:17 2020 -0700

[SPARK-31889][BUILD] Docker release script does not allocate enough memory 
to reliably publish

### What changes were proposed in this pull request?
Allow overriding the zinc options in the docker release and set a higher so 
the publish step can succeed consistently.

### Why are the changes needed?

The publish step experiences memory pressure.

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

No

### How was this patch tested?
Running test locally with fake user to see if publish step (besides svn 
part) succeeds

Closes #28698 from 
holdenk/SPARK-31889-docker-release-script-does-not-allocate-enough-memory-to-reliably-publish.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
(cherry picked from commit ab9e5a2fe9a0e9e93301096a027d7409fc3c9b64)
Signed-off-by: Holden Karau 
---
 dev/create-release/do-release-docker.sh | 1 +
 1 file changed, 1 insertion(+)

diff --git a/dev/create-release/do-release-docker.sh 
b/dev/create-release/do-release-docker.sh
index 2f794c0..8f53f4a 100755
--- a/dev/create-release/do-release-docker.sh
+++ b/dev/create-release/do-release-docker.sh
@@ -128,6 +128,7 @@ ASF_PASSWORD=$ASF_PASSWORD
 GPG_PASSPHRASE=$GPG_PASSPHRASE
 RELEASE_STEP=$RELEASE_STEP
 USER=$USER
+ZINC_OPTS=${RELEASE_ZINC_OPTS:-"-Xmx4g -XX:ReservedCodeCacheSize=2g"}
 EOF
 
 JAVA_VOL=


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



[spark] branch master updated: [SPARK-31889][BUILD] Docker release script does not allocate enough memory to reliably publish

2020-06-01 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden 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 ab9e5a2  [SPARK-31889][BUILD] Docker release script does not allocate 
enough memory to reliably publish
ab9e5a2 is described below

commit ab9e5a2fe9a0e9e93301096a027d7409fc3c9b64
Author: Holden Karau 
AuthorDate: Mon Jun 1 15:49:17 2020 -0700

[SPARK-31889][BUILD] Docker release script does not allocate enough memory 
to reliably publish

### What changes were proposed in this pull request?
Allow overriding the zinc options in the docker release and set a higher so 
the publish step can succeed consistently.

### Why are the changes needed?

The publish step experiences memory pressure.

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

No

### How was this patch tested?
Running test locally with fake user to see if publish step (besides svn 
part) succeeds

Closes #28698 from 
holdenk/SPARK-31889-docker-release-script-does-not-allocate-enough-memory-to-reliably-publish.

Authored-by: Holden Karau 
Signed-off-by: Holden Karau 
---
 dev/create-release/do-release-docker.sh | 1 +
 1 file changed, 1 insertion(+)

diff --git a/dev/create-release/do-release-docker.sh 
b/dev/create-release/do-release-docker.sh
index 2f794c0..8f53f4a 100755
--- a/dev/create-release/do-release-docker.sh
+++ b/dev/create-release/do-release-docker.sh
@@ -128,6 +128,7 @@ ASF_PASSWORD=$ASF_PASSWORD
 GPG_PASSPHRASE=$GPG_PASSPHRASE
 RELEASE_STEP=$RELEASE_STEP
 USER=$USER
+ZINC_OPTS=${RELEASE_ZINC_OPTS:-"-Xmx4g -XX:ReservedCodeCacheSize=2g"}
 EOF
 
 JAVA_VOL=


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



svn commit: r39857 - in /dev/spark/v2.4.6-rc8-docs: ./ _site/ _site/api/ _site/api/R/ _site/api/java/ _site/api/java/lib/ _site/api/java/org/ _site/api/java/org/apache/ _site/api/java/org/apache/spark

2020-05-29 Thread holden
Author: holden
Date: Sat May 30 01:27:54 2020
New Revision: 39857

Log:
Apache Spark v2.4.6-rc8 docs


[This commit notification would consist of 1458 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: r39856 - /dev/spark/v2.4.6-rc8-bin/

2020-05-29 Thread holden
Author: holden
Date: Sat May 30 00:54:32 2020
New Revision: 39856

Log:
Apache Spark v2.4.6-rc8

Added:
dev/spark/v2.4.6-rc8-bin/
dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz   (with props)
dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz.asc
dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz.sha512
dev/spark/v2.4.6-rc8-bin/pyspark-2.4.6.tar.gz   (with props)
dev/spark/v2.4.6-rc8-bin/pyspark-2.4.6.tar.gz.asc
dev/spark/v2.4.6-rc8-bin/pyspark-2.4.6.tar.gz.sha512
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-hadoop2.6.tgz   (with props)
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-hadoop2.6.tgz.asc
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-hadoop2.6.tgz.sha512
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-hadoop2.7.tgz   (with props)
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-hadoop2.7.tgz.asc
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-hadoop2.7.tgz.sha512
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-without-hadoop-scala-2.12.tgz   
(with props)
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-without-hadoop-scala-2.12.tgz.asc

dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-without-hadoop-scala-2.12.tgz.sha512
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-without-hadoop.tgz   (with props)
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-without-hadoop.tgz.asc
dev/spark/v2.4.6-rc8-bin/spark-2.4.6-bin-without-hadoop.tgz.sha512
dev/spark/v2.4.6-rc8-bin/spark-2.4.6.tgz   (with props)
dev/spark/v2.4.6-rc8-bin/spark-2.4.6.tgz.asc
dev/spark/v2.4.6-rc8-bin/spark-2.4.6.tgz.sha512

Added: dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz.asc
==
--- dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz.asc (added)
+++ dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz.asc Sat May 30 00:54:32 2020
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+Version: GnuPG v1
+
+iQIcBAABAgAGBQJe0abqAAoJEMQatnRCT8P7kGEP/2bc9UCJnW49GL1LGgXGca+a
+/88T3iO1FEAeJtukLYjK90nx/iAa0Sf7hwNWYm+tycbueJq5+SIwQYbe97jrs65T
+8NZeVasGhPflZ3W4beZqQohipPgIApPcdQIk40Uwj0pTEXsOZn9AiZwyRQ/HRUaI
+0alkUQc+18hm32dGIQPYAl3aOclxh9pSR2Nf4bpVYnkJWNXP/YIpWD69KGZHoRcS
+Cib4OS+VSG7FtpHV45W6g3Z0sjf9Ls8N1x0/14lWgMkuzwkpC6s0FRUcQ7n6LjWb
+ytYEupdLar0T2YGuL5gbJheF2SdjdB02jAdKB3utAKD0pIvYvHYJy+vZoYkt3hsG
+YjWWrnKMuP0k6jBUd3VSgsiSFCRe5OiZi6qUyg1va6ag7Y7qEarUT4SyLmH1EHG0
+kwcbNUW4D26KkUgxeXqBEUX9rtmVGyOGhy9T2BDKm1MlDPvCHLimPdCAumU6kLcy
+HSS3S2Pd2EX6znIdJlHASbTs7HGuhJ38CtYbGWTuf/X1OwqDwWBA1cA9IOMWcPx0
+xZCBDgIjtrtuB7IdxFN+yZcLD5peTDcI5/e7q8FEbovKdO4pKhQkDqvDoWSbVVC7
+JqWMG7oYgN3IOO7B8gpxbr/n/KWJZ8UWRrhJj7aRHdSSx45NjLVXfJWBGic8FGAZ
+5QQUa9SJZ+Zx2bjtL3/a
+=ZTPX
+-END PGP SIGNATURE-

Added: dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz.sha512
==
--- dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz.sha512 (added)
+++ dev/spark/v2.4.6-rc8-bin/SparkR_2.4.6.tar.gz.sha512 Sat May 30 00:54:32 2020
@@ -0,0 +1,3 @@
+SparkR_2.4.6.tar.gz: 3CC0F7FF FF0D6BD2 0A268A36 D9F33E42 49A66B56 4C49E834
+ 368DA707 E5AC3E5E 44B0A527 798181EF 4A951AAE 6E927206
+ 7246406F 7D7A38F6 BC059661 DA903604

Added: dev/spark/v2.4.6-rc8-bin/pyspark-2.4.6.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v2.4.6-rc8-bin/pyspark-2.4.6.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v2.4.6-rc8-bin/pyspark-2.4.6.tar.gz.asc
==
--- dev/spark/v2.4.6-rc8-bin/pyspark-2.4.6.tar.gz.asc (added)
+++ dev/spark/v2.4.6-rc8-bin/pyspark-2.4.6.tar.gz.asc Sat May 30 00:54:32 2020
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+Version: GnuPG v1
+
+iQIcBAABAgAGBQJe0aKWAAoJEMQatnRCT8P7N7EP/RuobQLVGHz6CpWFDKZCHUvk
+W24KFvAkhNS3S6+YzC5+GjZRM6WWQUx7n2+fwB7ntUq4B1IMZ9Tk5w3aVvmJkiOZ
+cwz1sqvcxmDZ97b0e8H/7j0Uue1V65nkVVnd9lrUgZicAaBRCkYJQdEqtUmJK+fd
+Wm4+9XHrssac1mG75b1DeEfjw9Ip3DhG+ey/QXZ1F7XaQcbnX36BKGi2PQhXF8No
+fFDDonnHxKh/JHvGyCCe6nixO4R6un28uTdUf5WXTToYrgsrp0Dn+N0bt+O5PMw3
+PcI7ODnsLVXgVsxkYZvhSq8Dskx6H8JYVTe4bv3Ayt2OfhNbbi1VPTnCKGeoyzxo
+7SKnmHLXeSqQ9AxhNRT+VxtR4GvJXNHT+MAFYwdtZQ0C8GqXqBlRfyUKkiSLaC5r
+8usaVtugOQRpEVepvSiwnsJngMIYUOhitBttRoEfpO6aikMsaCWk47s1CUpkkffO
+z7IqtaYwNXwUcRbk7ObUsep4hWfCGcPvwbDiRgvygOJ5LtMYM8foi3xNisw8OPU7
+V/VvO4jxPVroivHmokW9cZpaN3gV1tWNmMxrLPNJX0DCU41lVe+LZXi71Uf3sOmA
+HV7fH+bkMPzHxNog1uhO/8S3PvDwOWqoVStTJ116vKGokDttcr+1bb/mroYIb9Py
+4bhiZlciLcwLnfV//H0k
+=FPuQ
+-END PGP SIGNATURE-

Added: dev

[spark] 01/01: Preparing development version 2.4.7-SNAPSHOT

2020-05-29 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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

commit f32c60e001abe564736e0d00772dfce23c3b07b5
Author: Holden Karau 
AuthorDate: Fri May 29 23:28:37 2020 +

Preparing development version 2.4.7-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   | 4 ++--
 examples/pom.xml   | 2 +-
 external/avro/pom.xml  | 2 +-
 external/docker-integration-tests/pom.xml  | 2 +-
 external/flume-assembly/pom.xml| 2 +-
 external/flume-sink/pom.xml| 2 +-
 external/flume/pom.xml | 2 +-
 external/kafka-0-10-assembly/pom.xml   | 2 +-
 external/kafka-0-10-sql/pom.xml| 2 +-
 external/kafka-0-10/pom.xml| 2 +-
 external/kafka-0-8-assembly/pom.xml| 2 +-
 external/kafka-0-8/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 +-
 43 files changed, 44 insertions(+), 44 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index c913a38..b70014d 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 2.4.6
+Version: 2.4.7
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"),
diff --git a/assembly/pom.xml b/assembly/pom.xml
index de59f40..712cc7f 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index b3a2265..825d771 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index d3db527..9dd26b3 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 34872fd..386782b 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 5059ee0..8496a68 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/sketch

[spark] branch branch-2.4 updated (8307f1a -> f32c60e)

2020-05-29 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


from 8307f1a  [SPARK-26095][BUILD] Disable parallelization in 
make-distibution.sh.
 add 807e0a4  Preparing Spark release v2.4.6-rc8
 new f32c60e  Preparing development version 2.4.7-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:


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



[spark] 01/01: Preparing Spark release v2.4.6-rc8

2020-05-29 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a commit to tag v2.4.6-rc8
in repository https://gitbox.apache.org/repos/asf/spark.git

commit 807e0a484d1de767d1f02bd8a622da6450bdf940
Author: Holden Karau 
AuthorDate: Fri May 29 23:28:32 2020 +

Preparing Spark release v2.4.6-rc8
---
 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   | 4 ++--
 examples/pom.xml   | 2 +-
 external/avro/pom.xml  | 2 +-
 external/docker-integration-tests/pom.xml  | 2 +-
 external/flume-assembly/pom.xml| 2 +-
 external/flume-sink/pom.xml| 2 +-
 external/flume/pom.xml | 2 +-
 external/kafka-0-10-assembly/pom.xml   | 2 +-
 external/kafka-0-10-sql/pom.xml| 2 +-
 external/kafka-0-10/pom.xml| 2 +-
 external/kafka-0-8-assembly/pom.xml| 2 +-
 external/kafka-0-8/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 +-
 43 files changed, 44 insertions(+), 44 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index b70014d..c913a38 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 2.4.7
+Version: 2.4.6
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"),
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 712cc7f..de59f40 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.7-SNAPSHOT
+2.4.6
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 825d771..b3a2265 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.7-SNAPSHOT
+2.4.6
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index 9dd26b3..d3db527 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.7-SNAPSHOT
+2.4.6
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 386782b..34872fd 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.7-SNAPSHOT
+2.4.6
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 8496a68..5059ee0 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.7-SNAPSHOT
+2.4.6
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml 

[spark] tag v2.4.6-rc8 created (now 807e0a4)

2020-05-29 Thread holden
This is an automated email from the ASF dual-hosted git repository.

holden pushed a change to tag v2.4.6-rc8
in repository https://gitbox.apache.org/repos/asf/spark.git.


  at 807e0a4  (commit)
This tag includes the following new commits:

 new 807e0a4  Preparing Spark release v2.4.6-rc8

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 branch-2.4 updated: [SPARK-26095][BUILD] Disable parallelization in make-distibution.sh.

2020-05-29 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-2.4 by this push:
 new 8307f1a  [SPARK-26095][BUILD] Disable parallelization in 
make-distibution.sh.
8307f1a is described below

commit 8307f1aa6c421669367f60b1bc9a9d497fb74d6f
Author: Marcelo Vanzin 
AuthorDate: Fri Nov 16 15:57:38 2018 -0800

[SPARK-26095][BUILD] Disable parallelization in make-distibution.sh.

It makes the build slower, but at least it doesn't hang. Seems that
maven-shade-plugin has some issue with parallelization.

Closes #23061 from vanzin/SPARK-26095.

Authored-by: Marcelo Vanzin 
Signed-off-by: Marcelo Vanzin 
(cherry picked from commit d2792046a1b10a07b65fc30be573983f1237e450)
Signed-off-by: Holden Karau 
---
 dev/make-distribution.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh
index 78b85bd..2b02c24 100755
--- a/dev/make-distribution.sh
+++ b/dev/make-distribution.sh
@@ -166,7 +166,7 @@ export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g 
-XX:ReservedCodeCacheSize=1g}"
 # Store the command as an array because $MVN variable might have spaces in it.
 # Normal quoting tricks don't work.
 # See: http://mywiki.wooledge.org/BashFAQ/050
-BUILD_COMMAND=("$MVN" -T 1C clean package -DskipTests $@)
+BUILD_COMMAND=("$MVN" clean package -DskipTests $@)
 
 # Actually build the jar
 echo -e "\nBuilding with..."


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



svn commit: r39852 - in /dev/spark: v2.4.6-rc1-bin/ v2.4.6-rc1-docs/ v2.4.6-rc3-bin/ v2.4.6-rc3-docs/ v2.4.6-rc6-bin/ v2.4.6-rc6-docs/ v2.4.6-rc7-bin/

2020-05-29 Thread holden
Author: holden
Date: Fri May 29 19:49:01 2020
New Revision: 39852

Log:
Remove failed 2.4.6 RCs

Removed:
dev/spark/v2.4.6-rc1-bin/
dev/spark/v2.4.6-rc1-docs/
dev/spark/v2.4.6-rc3-bin/
dev/spark/v2.4.6-rc3-docs/
dev/spark/v2.4.6-rc6-bin/
dev/spark/v2.4.6-rc6-docs/
dev/spark/v2.4.6-rc7-bin/


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



svn commit: r39820 - /dev/spark/v2.4.6-rc7-bin/

2020-05-29 Thread holden
Author: holden
Date: Fri May 29 08:53:39 2020
New Revision: 39820

Log:
Apache Spark v2.4.6-rc7

Added:
dev/spark/v2.4.6-rc7-bin/
dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz   (with props)
dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz.asc
dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz.sha512
dev/spark/v2.4.6-rc7-bin/pyspark-2.4.6.tar.gz   (with props)
dev/spark/v2.4.6-rc7-bin/pyspark-2.4.6.tar.gz.asc
dev/spark/v2.4.6-rc7-bin/pyspark-2.4.6.tar.gz.sha512
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-hadoop2.6.tgz   (with props)
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-hadoop2.6.tgz.asc
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-hadoop2.6.tgz.sha512
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-hadoop2.7.tgz   (with props)
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-hadoop2.7.tgz.asc
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-hadoop2.7.tgz.sha512
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-without-hadoop-scala-2.12.tgz   
(with props)
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-without-hadoop-scala-2.12.tgz.asc

dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-without-hadoop-scala-2.12.tgz.sha512
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-without-hadoop.tgz   (with props)
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-without-hadoop.tgz.asc
dev/spark/v2.4.6-rc7-bin/spark-2.4.6-bin-without-hadoop.tgz.sha512
dev/spark/v2.4.6-rc7-bin/spark-2.4.6.tgz   (with props)
dev/spark/v2.4.6-rc7-bin/spark-2.4.6.tgz.asc
dev/spark/v2.4.6-rc7-bin/spark-2.4.6.tgz.sha512

Added: dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz.asc
==
--- dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz.asc (added)
+++ dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz.asc Fri May 29 08:53:39 2020
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+Version: GnuPG v1
+
+iQIcBAABAgAGBQJe0MgaAAoJEMQatnRCT8P79NEP/1zQxuI6gVcD6gsNMoYF8k+c
+woyDyHnNcEBZsTL7WTPu/ut/hrCe8anXVfzzsc6LudB5cRRbvO1KXpYvcARmKQID
+tnoUZLVUuFs+ejFsM+Ux2vTvrF06wPxWBKXHw71QYU4hVKnl2X8oekK1Ae2u74Yl
+RCtTzjbbgxYc5NyvKBnMU2lrxij+izFunQy3UGPuvxhwS0Ip2VWId04jWGwE1hA0
+S+UoEG8aZI3R7by//AO8+lMhRl2ZOKCn0Le94EapT1R3bBCF5kIdC82Q6xR5z4Z+
+3C8lc+sNO7BJmscJeMXIs5JRHVb/twdvTnPs2HIQ0hZQUaT3GrByaE/lJkeA4rcg
+qewhnCBX/S/E4spETwB9nJjFyt2caT1L06nW7ICK2zXiTmbPiB2jTWh+v7KDmdNN
+8dMr0lvbvmNAzKwBFEthremEEKoqp82dImMffTlJSMGmnAoCgfBDB7OuuP1Pq7G5
+KZnY7f6BA6ylVp2gxGHCHTOYEJd3fR4fHSFv2AUOyh96HZ3yqjFP3bSn2PEMNvfH
+2C2E7UG7cNgdHef6IQXRktxyvoWDN1tvhVh5AJU1hBFJ4bCD16SsJls/cvvn4AMn
+Sa43Yt6YPzeKpJh2xIkhWW7xK2EnjDwT1KXx819M+qmatKANPcoZpHMqBLuXap8g
+FC/l5MpURmJtItJCUfm0
+=/74N
+-END PGP SIGNATURE-

Added: dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz.sha512
==
--- dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz.sha512 (added)
+++ dev/spark/v2.4.6-rc7-bin/SparkR_2.4.6.tar.gz.sha512 Fri May 29 08:53:39 2020
@@ -0,0 +1,3 @@
+SparkR_2.4.6.tar.gz: 664DE4CC CBA415FA CBCEF3B0 2D6B83B4 D44D4AA6 B0A9A3B5
+ D27DBBB8 5889858C 6FCC2394 C5B6BAE9 1B5546DF 44EB0512
+ DDEDC106 52763F02 0AF834C0 C091542B

Added: dev/spark/v2.4.6-rc7-bin/pyspark-2.4.6.tar.gz
==
Binary file - no diff available.

Propchange: dev/spark/v2.4.6-rc7-bin/pyspark-2.4.6.tar.gz
--
svn:mime-type = application/octet-stream

Added: dev/spark/v2.4.6-rc7-bin/pyspark-2.4.6.tar.gz.asc
==
--- dev/spark/v2.4.6-rc7-bin/pyspark-2.4.6.tar.gz.asc (added)
+++ dev/spark/v2.4.6-rc7-bin/pyspark-2.4.6.tar.gz.asc Fri May 29 08:53:39 2020
@@ -0,0 +1,17 @@
+-BEGIN PGP SIGNATURE-
+Version: GnuPG v1
+
+iQIcBAABAgAGBQJe0MTQAAoJEMQatnRCT8P7EcgP/3LOQmy+nmMLhdllrzWvYbUD
+jl01lpIKVYWFhF1w9sXZ76UjzKnuj56gwsVlXUkL6k56GVQEcP6Vyu7pOCflwAZz
+nQbCkFZFnajFeBJ/3qKu6iqsIDEcvDt5vSIwNw+CdIzM/Jc4evpQ2Lqxj4/eUhTf
+laSj6Qilj2OiMdRpr/UIWpbNN/JWlxhov6a9vHnWCkF5DYexuVuUzMVPNC8oWxWq
+adtfWS2/MzYB6Y/wYifnGEZz1MOZysTHEprvx8CiFPznb00a224dZ8W3P1emH5dX
++tQjuFcg4TDpRKPqO9nLAkASYRso0kuhwMVLHh+Xoq1GqOhMCTMMx+uUk0L8NOJJ
+KNUjUaSUIR4UsWA30aEWvWV+a1xhduhWSlj7rgylPj+slxfO1aAjiBkSK8DfGz3U
+3fSsX8r4l7dZOjPDg0/6EnlI6RDbReZ30MGHQ8FBKGuZ9SezHPMpqN2/ny6pOMTM
+RhgXCgNbLcolp6satIHbgWz88bFv3cbBoWdQYrN7nqbqc8fXaGhjeaw50Et5IG77
+QylBnqX424xVFP7G+1CFpcuxgsXUGXhLSLqCQNchBbGVR/IXXL6skJUgyXL3xz6t
+LpnTWnKm7xdFaaWOG9rp82yZVQSti2Feh36c0NwXjrk1BwjFiW0Jj9p1MEAWmlrF
+G18Qlh8n7RUUqk67SQK/
+=H4tm
+-END PGP SIGNATURE-

Added: dev

[spark] 01/01: Preparing development version 2.4.7-SNAPSHOT

2020-05-29 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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

commit 1c36f8e8f633cce8c78bccdac215acccd2d8202b
Author: Holden Karau 
AuthorDate: Fri May 29 07:51:23 2020 +

Preparing development version 2.4.7-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   | 4 ++--
 examples/pom.xml   | 2 +-
 external/avro/pom.xml  | 2 +-
 external/docker-integration-tests/pom.xml  | 2 +-
 external/flume-assembly/pom.xml| 2 +-
 external/flume-sink/pom.xml| 2 +-
 external/flume/pom.xml | 2 +-
 external/kafka-0-10-assembly/pom.xml   | 2 +-
 external/kafka-0-10-sql/pom.xml| 2 +-
 external/kafka-0-10/pom.xml| 2 +-
 external/kafka-0-8-assembly/pom.xml| 2 +-
 external/kafka-0-8/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 +-
 43 files changed, 44 insertions(+), 44 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index c913a38..b70014d 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 2.4.6
+Version: 2.4.7
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
<https://spark.apache.org>.
 Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"),
diff --git a/assembly/pom.xml b/assembly/pom.xml
index de59f40..712cc7f 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index b3a2265..825d771 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index d3db527..9dd26b3 100644
--- a/common/network-common/pom.xml
+++ b/common/network-common/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 34872fd..386782b 100644
--- a/common/network-shuffle/pom.xml
+++ b/common/network-shuffle/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 5059ee0..8496a68 100644
--- a/common/network-yarn/pom.xml
+++ b/common/network-yarn/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.11
-2.4.6
+2.4.7-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/sketch

[spark] branch branch-2.4 updated (d53363d -> 1c36f8e)

2020-05-29 Thread holden
This is an automated email from the ASF dual-hosted git repository.

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


from d53363d  Preparing development version 2.4.7-SNAPSHOT
 add 105de0512 Preparing Spark release v2.4.6-rc7
 new 1c36f8e  Preparing development version 2.4.7-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:


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



  1   2   3   4   5   >