[spark] branch master updated: [SPARK-38324][SQL] The second range is not [0, 59] in the day time ANSI interval

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

maxgekk 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 9f7582c8cbe [SPARK-38324][SQL] The second range is not [0, 59] in the 
day time ANSI interval
9f7582c8cbe is described below

commit 9f7582c8cbeae70c31c183567bf5320d1c3210fe
Author: haoyan.zhang 
AuthorDate: Fri Feb 17 09:29:42 2023 +0300

[SPARK-38324][SQL] The second range is not [0, 59] in the day time ANSI 
interval

### What changes were proposed in this pull request?
Throw an error when the second value in day(hour, minute) to second 
interval out of range [0, 59]

### Why are the changes needed?
Currently an invalid second value will not get an error
```
spark.sql("select INTERVAL '10 01:01:99' DAY TO SECOND")
DataFrame[INTERVAL '10 01:02:39' DAY TO SECOND: interval day to second]{}
```
But minute range check is ok
```
spark.sql("select INTERVAL '10 01:60:01' DAY TO SECOND")
requirement failed: minute 60 outside range [0, 59](line 1, pos 16)
```

### We need check second value too

Does this PR introduce any user-facing change?
no

### How was this patch tested?
New unit tests.

Closes #40033 from haoyanzhang/master.

Authored-by: haoyan.zhang 
Signed-off-by: Max Gekk 
---
 .../apache/spark/sql/catalyst/util/IntervalUtils.scala | 13 -
 .../spark/sql/catalyst/util/IntervalUtilsSuite.scala   | 18 ++
 2 files changed, 26 insertions(+), 5 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
index 7b574e987d9..455a74e06c4 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala
@@ -381,7 +381,7 @@ object IntervalUtils {
 micros = Math.addExact(micros, sign * hours * MICROS_PER_HOUR)
 val minutes = toLongWithRange(minuteStr, minute, 0, 59)
 micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE)
-micros = Math.addExact(micros, sign * parseSecondNano(second))
+micros = Math.addExact(micros, sign * parseSecondNano(second, 0, 59))
 micros
   }
 
@@ -391,7 +391,7 @@ object IntervalUtils {
 micros = Math.addExact(micros, sign * hours * MICROS_PER_HOUR)
 val minutes = toLongWithRange(minuteStr, minute, 0, 59)
 micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE)
-micros = Math.addExact(micros, sign * parseSecondNano(second))
+micros = Math.addExact(micros, sign * parseSecondNano(second, 0, 59))
 micros
   }
 
@@ -399,7 +399,7 @@ object IntervalUtils {
 var micros = 0L
 val minutes = toLongWithRange(minuteStr, minute, 0, MAX_MINUTE)
 micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE)
-micros = Math.addExact(micros, sign * parseSecondNano(second))
+micros = Math.addExact(micros, sign * parseSecondNano(second, 0, 59))
 micros
   }
 
@@ -549,9 +549,12 @@ object IntervalUtils {
   /**
* Parse second_nano string in ss.n format to microseconds
*/
-  private def parseSecondNano(secondNano: String): Long = {
+  private def parseSecondNano(
+  secondNano: String,
+  minSecond: Long = MIN_SECOND,
+  maxSecond: Long = MAX_SECOND): Long = {
 def parseSeconds(secondsStr: String): Long = {
-  toLongWithRange(secondStr, secondsStr, MIN_SECOND, MAX_SECOND) * 
MICROS_PER_SECOND
+  toLongWithRange(secondStr, secondsStr, minSecond, maxSecond) * 
MICROS_PER_SECOND
 }
 
 secondNano.split("\\.") match {
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala
index 0e65886a2eb..3ba6787045e 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/IntervalUtilsSuite.scala
@@ -665,4 +665,22 @@ class IntervalUtilsSuite extends SparkFunSuite with 
SQLHelper {
   assert(toYearMonthIntervalString(months, ANSI_STYLE, MONTH, MONTH) === 
month)
 }
   }
+
+  test("SPARK-38324: The second range is not [0, 59] in the day time ANSI 
interval") {
+import org.apache.spark.sql.types.DayTimeIntervalType._
+Seq(
+  ("10 12:40:60", 60, DAY, SECOND),
+  ("10 12:40:60.9", 60, DAY, SECOND),
+  ("10 12:40:99.9", 99, DAY, SECOND),
+  ("12:40:60", 60, HOUR, SECOND),
+  ("12:40:60.9", 60, HOUR, SECOND),
+  ("12:40:99.9", 99, HOUR, SECOND),
+  ("40:60", 60, MINUTE, SECOND),
+  ("40:60.9", 60, MINUTE, SECOND),
+  

[spark] branch branch-3.4 updated: [SPARK-39859][SQL] Support v2 DESCRIBE TABLE EXTENDED for columns

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new ca811db31f9 [SPARK-39859][SQL] Support v2 DESCRIBE TABLE EXTENDED for 
columns
ca811db31f9 is described below

commit ca811db31f983ce75a92c2d918e334610c0f7632
Author: huaxingao 
AuthorDate: Thu Feb 16 20:40:58 2023 -0800

[SPARK-39859][SQL] Support v2 DESCRIBE TABLE EXTENDED for columns

### What changes were proposed in this pull request?
Support v2 DESCRIBE TABLE EXTENDED for columns

### Why are the changes needed?
DS v1/v2 command parity

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

### How was this patch tested?
UT

Closes #40058 from huaxingao/describe_col.

Authored-by: huaxingao 
Signed-off-by: Dongjoon Hyun 
(cherry picked from commit ebab0ef7c8572e1dac41474c5991f482dbe9d253)
Signed-off-by: Dongjoon Hyun 
---
 .../datasources/v2/DataSourceV2Strategy.scala  | 13 +--
 .../datasources/v2/DescribeColumnExec.scala| 42 --
 .../execution/command/v2/DescribeTableSuite.scala  | 11 --
 3 files changed, 59 insertions(+), 7 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
index 29f0da1158f..757b66e1534 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
@@ -36,7 +36,7 @@ import org.apache.spark.sql.connector.catalog.{Identifier, 
StagingTableCatalog,
 import org.apache.spark.sql.connector.catalog.index.SupportsIndex
 import org.apache.spark.sql.connector.expressions.{FieldReference, 
LiteralValue}
 import org.apache.spark.sql.connector.expressions.filter.{And => V2And, Not => 
V2Not, Or => V2Or, Predicate}
-import org.apache.spark.sql.connector.read.LocalScan
+import org.apache.spark.sql.connector.read.{LocalScan, 
SupportsReportStatistics}
 import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, 
MicroBatchStream}
 import org.apache.spark.sql.connector.write.V1Write
 import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
@@ -329,10 +329,17 @@ class DataSourceV2Strategy(session: SparkSession) extends 
Strategy with Predicat
   }
   DescribeTableExec(output, r.table, isExtended) :: Nil
 
-case DescribeColumn(_: ResolvedTable, column, isExtended, output) =>
+case DescribeColumn(r: ResolvedTable, column, isExtended, output) =>
   column match {
 case c: Attribute =>
-  DescribeColumnExec(output, c, isExtended) :: Nil
+  val colStats =
+
r.table.asReadable.newScanBuilder(CaseInsensitiveStringMap.empty()).build() 
match {
+case s: SupportsReportStatistics =>
+  val stats = s.estimateStatistics()
+  Some(stats.columnStats().get(FieldReference.column(c.name)))
+case _ => None
+  }
+  DescribeColumnExec(output, c, isExtended, colStats) :: Nil
 case nested =>
   throw QueryCompilationErrors.commandNotSupportNestedColumnError(
 "DESC TABLE COLUMN", toPrettySQL(nested))
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
index 3be9b5c5471..491c214080a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
@@ -22,11 +22,13 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.read.colstats.ColumnStatistics
 
 case class DescribeColumnExec(
 override val output: Seq[Attribute],
 column: Attribute,
-isExtended: Boolean) extends LeafV2CommandExec {
+isExtended: Boolean,
+colStats: Option[ColumnStatistics] = None) extends LeafV2CommandExec {
 
   override protected def run(): Seq[InternalRow] = {
 val rows = new ArrayBuffer[InternalRow]()
@@ -42,7 +44,43 @@ case class DescribeColumnExec(
   
CharVarcharUtils.getRawType(column.metadata).getOrElse(column.dataType).catalogString)
 rows += toCatalystRow("comment", comment)
 
-// TODO: The extended description (isExtended = true) can be added here.
+if (isExtended && colStats.nonEmpty) {
+  if (colStats.get.min().isPresent) {

[spark] branch master updated: [SPARK-39859][SQL] Support v2 DESCRIBE TABLE EXTENDED for columns

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

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


The following commit(s) were added to refs/heads/master by this push:
 new ebab0ef7c85 [SPARK-39859][SQL] Support v2 DESCRIBE TABLE EXTENDED for 
columns
ebab0ef7c85 is described below

commit ebab0ef7c8572e1dac41474c5991f482dbe9d253
Author: huaxingao 
AuthorDate: Thu Feb 16 20:40:58 2023 -0800

[SPARK-39859][SQL] Support v2 DESCRIBE TABLE EXTENDED for columns

### What changes were proposed in this pull request?
Support v2 DESCRIBE TABLE EXTENDED for columns

### Why are the changes needed?
DS v1/v2 command parity

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

### How was this patch tested?
UT

Closes #40058 from huaxingao/describe_col.

Authored-by: huaxingao 
Signed-off-by: Dongjoon Hyun 
---
 .../datasources/v2/DataSourceV2Strategy.scala  | 13 +--
 .../datasources/v2/DescribeColumnExec.scala| 42 --
 .../execution/command/v2/DescribeTableSuite.scala  | 11 --
 3 files changed, 59 insertions(+), 7 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
index 29f0da1158f..757b66e1534 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
@@ -36,7 +36,7 @@ import org.apache.spark.sql.connector.catalog.{Identifier, 
StagingTableCatalog,
 import org.apache.spark.sql.connector.catalog.index.SupportsIndex
 import org.apache.spark.sql.connector.expressions.{FieldReference, 
LiteralValue}
 import org.apache.spark.sql.connector.expressions.filter.{And => V2And, Not => 
V2Not, Or => V2Or, Predicate}
-import org.apache.spark.sql.connector.read.LocalScan
+import org.apache.spark.sql.connector.read.{LocalScan, 
SupportsReportStatistics}
 import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, 
MicroBatchStream}
 import org.apache.spark.sql.connector.write.V1Write
 import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
@@ -329,10 +329,17 @@ class DataSourceV2Strategy(session: SparkSession) extends 
Strategy with Predicat
   }
   DescribeTableExec(output, r.table, isExtended) :: Nil
 
-case DescribeColumn(_: ResolvedTable, column, isExtended, output) =>
+case DescribeColumn(r: ResolvedTable, column, isExtended, output) =>
   column match {
 case c: Attribute =>
-  DescribeColumnExec(output, c, isExtended) :: Nil
+  val colStats =
+
r.table.asReadable.newScanBuilder(CaseInsensitiveStringMap.empty()).build() 
match {
+case s: SupportsReportStatistics =>
+  val stats = s.estimateStatistics()
+  Some(stats.columnStats().get(FieldReference.column(c.name)))
+case _ => None
+  }
+  DescribeColumnExec(output, c, isExtended, colStats) :: Nil
 case nested =>
   throw QueryCompilationErrors.commandNotSupportNestedColumnError(
 "DESC TABLE COLUMN", toPrettySQL(nested))
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
index 3be9b5c5471..491c214080a 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala
@@ -22,11 +22,13 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.read.colstats.ColumnStatistics
 
 case class DescribeColumnExec(
 override val output: Seq[Attribute],
 column: Attribute,
-isExtended: Boolean) extends LeafV2CommandExec {
+isExtended: Boolean,
+colStats: Option[ColumnStatistics] = None) extends LeafV2CommandExec {
 
   override protected def run(): Seq[InternalRow] = {
 val rows = new ArrayBuffer[InternalRow]()
@@ -42,7 +44,43 @@ case class DescribeColumnExec(
   
CharVarcharUtils.getRawType(column.metadata).getOrElse(column.dataType).catalogString)
 rows += toCatalystRow("comment", comment)
 
-// TODO: The extended description (isExtended = true) can be added here.
+if (isExtended && colStats.nonEmpty) {
+  if (colStats.get.min().isPresent) {
+rows += toCatalystRow("min", colStats.get.min().toString)
+  } else {
+rows += 

[spark] branch master updated: [SPARK-42469][SQL] Update MSSQL Dialect to use parentheses for TOP and add tests for Limit clause

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

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


The following commit(s) were added to refs/heads/master by this push:
 new 9d876844082 [SPARK-42469][SQL] Update MSSQL Dialect to use parentheses 
for TOP and add tests for Limit clause
9d876844082 is described below

commit 9d876844082d588a15e09b20f2cd229041ea556b
Author: Ivan Sadikov 
AuthorDate: Thu Feb 16 20:38:53 2023 -0800

[SPARK-42469][SQL] Update MSSQL Dialect to use parentheses for TOP and add 
tests for Limit clause

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

This PR is a follow-up for 
https://issues.apache.org/jira/browse/SPARK-42131. It adds parentheses in TOP 
clause in MSSQL dialect as they are only omitted for backward compatibility and 
required otherwise: 
https://learn.microsoft.com/en-us/sql/t-sql/queries/top-transact-sql?view=sql-server-ver16#compatibility-support.

I also added tests to check Limit clause translation for Oracle and MSSQL 
dialects.

### Why are the changes needed?

Updates TOP to include round brackets.

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

No.

### How was this patch tested?

I added unit tests.

Closes #40059 from sadikovi/jdbc-small-fix.

Authored-by: Ivan Sadikov 
Signed-off-by: Dongjoon Hyun 
---
 .../apache/spark/sql/jdbc/MsSqlServerDialect.scala |  2 +-
 .../org/apache/spark/sql/jdbc/JDBCSuite.scala  | 28 +-
 2 files changed, 28 insertions(+), 2 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
index 39b617135ce..156f495943e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala
@@ -183,7 +183,7 @@ private object MsSqlServerDialect extends JdbcDialect {
   }
 
   override def getLimitClause(limit: Integer): String = {
-if (limit > 0) s"TOP $limit" else ""
+if (limit > 0) s"TOP ($limit)" else ""
   }
 
   override def classifyException(message: String, e: Throwable): 
AnalysisException = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index aa66fcd5304..27609de5433 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -959,6 +959,19 @@ class JDBCSuite extends QueryTest with SharedSparkSession {
   Some(TimestampType))
   }
 
+  test("SPARK-42469: OracleDialect Limit query test") {
+// JDBC url is a required option but is not used in this test.
+val options = new JDBCOptions(Map("url" -> "jdbc:h2://host:port", 
"dbtable" -> "test"))
+assert(
+  OracleDialect
+.getJdbcSQLQueryBuilder(options)
+.withColumns(Array("a", "b"))
+.withLimit(123)
+.build()
+.trim() ==
+  "SELECT tab.* FROM (SELECT a,b FROM test) tab WHERE rownum <= 123")
+  }
+
   test("MsSqlServerDialect jdbc type mapping") {
 val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver")
 
assert(msSqlServerDialect.getJDBCType(TimestampType).map(_.databaseTypeDefinition).get
 ==
@@ -981,7 +994,7 @@ class JDBCSuite extends QueryTest with SharedSparkSession {
 }
   }
 
-  test("SPARK-28152 MsSqlServerDialect catalyst type mapping") {
+  test("SPARK-28152: MsSqlServerDialect catalyst type mapping") {
 val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver")
 val metadata = new MetadataBuilder().putLong("scale", 1)
 
@@ -1002,6 +1015,19 @@ class JDBCSuite extends QueryTest with 
SharedSparkSession {
 }
   }
 
+  test("SPARK-42469: MsSqlServerDialect Limit query test") {
+// JDBC url is a required option but is not used in this test.
+val options = new JDBCOptions(Map("url" -> "jdbc:h2://host:port", 
"dbtable" -> "test"))
+assert(
+  MsSqlServerDialect
+.getJdbcSQLQueryBuilder(options)
+.withColumns(Array("a", "b"))
+.withLimit(123)
+.build()
+.trim() ==
+  "SELECT TOP (123) a,b FROM test")
+  }
+
   test("table exists query by jdbc dialect") {
 val MySQL = JdbcDialects.get("jdbc:mysql://127.0.0.1/db")
 val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db")


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



[spark] branch branch-3.4 updated (7c1c8be960e -> 9f22acdc320)

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

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


from 7c1c8be960e [SPARK-42468][CONNECT] Implement agg by (String, String)*
 add d89d6abd83a Preparing Spark release v3.4.0-rc1
 new 9f22acdc320 Preparing development version 3.4.1-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 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 6 +++---
 examples/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, 45 insertions(+), 45 deletions(-)


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

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

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

commit 9f22acdc32055d5437f43fa0f77644433ec32822
Author: Xinrong Meng 
AuthorDate: Fri Feb 17 03:54:24 2023 +

Preparing development version 3.4.1-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 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 6 +++---
 examples/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, 45 insertions(+), 45 deletions(-)

diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index 4a32762b34c..fa7028630a8 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -1,6 +1,6 @@
 Package: SparkR
 Type: Package
-Version: 3.4.0
+Version: 3.4.1
 Title: R Front End for 'Apache Spark'
 Description: Provides an R Front end for 'Apache Spark' 
.
 Authors@R:
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 58dd9ef46e0..a4111eb64d9 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0
+3.4.1-SNAPSHOT
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index 95ea15552da..f9ecfb3d692 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0
+3.4.1-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index e4d98471bf9..22ee65b7d25 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.4.0
+3.4.1-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index 7a6d5aedf65..2c67da81ca4 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.4.0
+3.4.1-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index 1c421754083..219682e047d 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.4.0
+3.4.1-SNAPSHOT
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index 

[spark] 01/01: Preparing Spark release v3.4.0-rc1

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

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

commit d89d6abd83a51fcab75e0aea0a3d5ba943bd2280
Author: Xinrong Meng 
AuthorDate: Fri Feb 17 03:54:15 2023 +

Preparing Spark release v3.4.0-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 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 4 ++--
 examples/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 +-
 42 files changed, 43 insertions(+), 43 deletions(-)

diff --git a/assembly/pom.xml b/assembly/pom.xml
index f37edcd7e49..58dd9ef46e0 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0-SNAPSHOT
+3.4.0
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index d19883549d0..95ea15552da 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index ff2cc71c2c9..e4d98471bf9 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.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index de4730c5b71..7a6d5aedf65 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.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index a77732bb8b8..1c421754083 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.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index f4e442b3ba7..2ee25ebfffc 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/common/tags/pom.xml
index 216812152ad..5c31e6eb365 100644
--- a/common/tags/pom.xml
+++ b/common/tags/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0-SNAPSHOT
+

[spark] tag v3.4.0-rc1 created (now d89d6abd83a)

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

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


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

 new d89d6abd83a Preparing Spark release v3.4.0-rc1

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



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



[spark] branch branch-3.4 updated: [SPARK-42468][CONNECT] Implement agg by (String, String)*

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 7c1c8be960e [SPARK-42468][CONNECT] Implement agg by (String, String)*
7c1c8be960e is described below

commit 7c1c8be960ed9b30451807b460ca45ca9ddf8a72
Author: Rui Wang 
AuthorDate: Thu Feb 16 23:02:51 2023 -0400

[SPARK-42468][CONNECT] Implement agg by (String, String)*

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

Starting to support basic aggregation in Scala client. The first step is to 
support aggregation by strings.

### Why are the changes needed?

API coverage

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

No

### How was this patch tested?

UT

Closes #40057 from amaliujia/rw-agg.

Authored-by: Rui Wang 
Signed-off-by: Herman van Hovell 
(cherry picked from commit cc471a52d162d0e4d4063372253ed06a62f5cb19)
Signed-off-by: Herman van Hovell 
---
 .../main/scala/org/apache/spark/sql/Dataset.scala  |  23 
 .../spark/sql/RelationalGroupedDataset.scala   | 152 +
 .../apache/spark/sql/PlanGenerationTestSuite.scala |  14 ++
 .../explain-results/groupby_agg.explain|   2 +
 .../resources/query-tests/queries/groupby_agg.json |  88 
 .../query-tests/queries/groupby_agg.proto.bin  |  19 +++
 6 files changed, 298 insertions(+)

diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
index 977c823f7c7..c39fc6100f5 100644
--- 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -1035,6 +1035,29 @@ class Dataset[T] private[sql] (val session: 
SparkSession, private[sql] val plan:
 }
   }
 
+  /**
+   * Groups the Dataset using the specified columns, so we can run aggregation 
on them. See
+   * [[RelationalGroupedDataset]] for all the available aggregate functions.
+   *
+   * {{{
+   *   // Compute the average for all numeric columns grouped by department.
+   *   ds.groupBy($"department").avg()
+   *
+   *   // Compute the max age and average salary, grouped by department and 
gender.
+   *   ds.groupBy($"department", $"gender").agg(Map(
+   * "salary" -> "avg",
+   * "age" -> "max"
+   *   ))
+   * }}}
+   *
+   * @group untypedrel
+   * @since 3.4.0
+   */
+  @scala.annotation.varargs
+  def groupBy(cols: Column*): RelationalGroupedDataset = {
+new RelationalGroupedDataset(toDF(), cols.map(_.expr))
+  }
+
   /**
* Unpivot a DataFrame from wide format to long format, optionally leaving 
identifier columns
* set. This is the reverse to `groupBy(...).pivot(...).agg(...)`, except 
for the aggregation,
diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala
new file mode 100644
index 000..a3dfcb01fdc
--- /dev/null
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.util.Locale
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto
+
+/**
+ * A set of methods for aggregations on a `DataFrame`, created by 
[[Dataset#groupBy groupBy]],
+ * [[Dataset#cube cube]] or [[Dataset#rollup rollup]] (and also `pivot`).
+ *
+ * The main method is the `agg` function, which has multiple variants. This 
class also contains
+ * some first-order statistics such as `mean`, `sum` for convenience.
+ *
+ * @note
+ *   This class was named `GroupedData` in Spark 1.x.
+ *
+ * @since 3.4.0
+ */
+class RelationalGroupedDataset protected[sql] (
+private[sql] val df: DataFrame,
+private[sql] val groupingExprs: Seq[proto.Expression]) {
+
+  private[this] 

[spark] branch master updated: [SPARK-42468][CONNECT] Implement agg by (String, String)*

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

hvanhovell 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 cc471a52d16 [SPARK-42468][CONNECT] Implement agg by (String, String)*
cc471a52d16 is described below

commit cc471a52d162d0e4d4063372253ed06a62f5cb19
Author: Rui Wang 
AuthorDate: Thu Feb 16 23:02:51 2023 -0400

[SPARK-42468][CONNECT] Implement agg by (String, String)*

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

Starting to support basic aggregation in Scala client. The first step is to 
support aggregation by strings.

### Why are the changes needed?

API coverage

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

No

### How was this patch tested?

UT

Closes #40057 from amaliujia/rw-agg.

Authored-by: Rui Wang 
Signed-off-by: Herman van Hovell 
---
 .../main/scala/org/apache/spark/sql/Dataset.scala  |  23 
 .../spark/sql/RelationalGroupedDataset.scala   | 152 +
 .../apache/spark/sql/PlanGenerationTestSuite.scala |  14 ++
 .../explain-results/groupby_agg.explain|   2 +
 .../resources/query-tests/queries/groupby_agg.json |  88 
 .../query-tests/queries/groupby_agg.proto.bin  |  19 +++
 6 files changed, 298 insertions(+)

diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
index 977c823f7c7..c39fc6100f5 100644
--- 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -1035,6 +1035,29 @@ class Dataset[T] private[sql] (val session: 
SparkSession, private[sql] val plan:
 }
   }
 
+  /**
+   * Groups the Dataset using the specified columns, so we can run aggregation 
on them. See
+   * [[RelationalGroupedDataset]] for all the available aggregate functions.
+   *
+   * {{{
+   *   // Compute the average for all numeric columns grouped by department.
+   *   ds.groupBy($"department").avg()
+   *
+   *   // Compute the max age and average salary, grouped by department and 
gender.
+   *   ds.groupBy($"department", $"gender").agg(Map(
+   * "salary" -> "avg",
+   * "age" -> "max"
+   *   ))
+   * }}}
+   *
+   * @group untypedrel
+   * @since 3.4.0
+   */
+  @scala.annotation.varargs
+  def groupBy(cols: Column*): RelationalGroupedDataset = {
+new RelationalGroupedDataset(toDF(), cols.map(_.expr))
+  }
+
   /**
* Unpivot a DataFrame from wide format to long format, optionally leaving 
identifier columns
* set. This is the reverse to `groupBy(...).pivot(...).agg(...)`, except 
for the aggregation,
diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala
new file mode 100644
index 000..a3dfcb01fdc
--- /dev/null
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.util.Locale
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.connect.proto
+
+/**
+ * A set of methods for aggregations on a `DataFrame`, created by 
[[Dataset#groupBy groupBy]],
+ * [[Dataset#cube cube]] or [[Dataset#rollup rollup]] (and also `pivot`).
+ *
+ * The main method is the `agg` function, which has multiple variants. This 
class also contains
+ * some first-order statistics such as `mean`, `sum` for convenience.
+ *
+ * @note
+ *   This class was named `GroupedData` in Spark 1.x.
+ *
+ * @since 3.4.0
+ */
+class RelationalGroupedDataset protected[sql] (
+private[sql] val df: DataFrame,
+private[sql] val groupingExprs: Seq[proto.Expression]) {
+
+  private[this] def toDF(aggExprs: Seq[proto.Expression]): DataFrame = {
+// TODO: support other GroupByType such as Rollup, Cube, 

[spark] branch branch-3.4 updated: [SPARK-42002][CONNECT][PYTHON][FOLLOWUP] Enable tests in ReadwriterV2ParityTests

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new d3a841d3ce8 [SPARK-42002][CONNECT][PYTHON][FOLLOWUP] Enable tests in 
ReadwriterV2ParityTests
d3a841d3ce8 is described below

commit d3a841d3ce8bc1638f3431b596e71fad07cb953b
Author: Takuya UESHIN 
AuthorDate: Fri Feb 17 09:26:12 2023 +0800

[SPARK-42002][CONNECT][PYTHON][FOLLOWUP] Enable tests in 
ReadwriterV2ParityTests

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

Enables tests in `ReadwriterV2ParityTests`.

### Why are the changes needed?

Now that `DataFrameWriterV2` for Spark Connect is implemented, we can 
enable tests in `ReadwriterV2ParityTests`.

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

No.

### How was this patch tested?

Enabled tests.

Closes #40060 from ueshin/issues/SPARK-42002/tests.

Authored-by: Takuya UESHIN 
Signed-off-by: Ruifeng Zheng 
(cherry picked from commit 961d5bd909744fea24e2391cd1a7aea3c96c418d)
Signed-off-by: Ruifeng Zheng 
---
 .../sql/tests/connect/test_parity_readwriter.py|  9 ++
 python/pyspark/sql/tests/test_readwriter.py| 36 --
 2 files changed, 23 insertions(+), 22 deletions(-)

diff --git a/python/pyspark/sql/tests/connect/test_parity_readwriter.py 
b/python/pyspark/sql/tests/connect/test_parity_readwriter.py
index 61f542385ec..bf77043ef38 100644
--- a/python/pyspark/sql/tests/connect/test_parity_readwriter.py
+++ b/python/pyspark/sql/tests/connect/test_parity_readwriter.py
@@ -16,6 +16,7 @@
 #
 import unittest
 
+from pyspark.sql.connect.readwriter import DataFrameWriterV2
 from pyspark.sql.tests.test_readwriter import ReadwriterTestsMixin, 
ReadwriterV2TestsMixin
 from pyspark.testing.connectutils import ReusedConnectTestCase
 
@@ -33,15 +34,11 @@ class ReadwriterParityTests(ReadwriterTestsMixin, 
ReusedConnectTestCase):
 
 
 class ReadwriterV2ParityTests(ReadwriterV2TestsMixin, ReusedConnectTestCase):
-# TODO(SPARK-42002): Implement writeTo()
-@unittest.skip("Fails in Spark Connect, should enable.")
 def test_api(self):
-super().test_api()
+self.check_api(DataFrameWriterV2)
 
-# TODO(SPARK-42002): Implement writeTo()
-@unittest.skip("Fails in Spark Connect, should enable.")
 def test_partitioning_functions(self):
-super().test_partitioning_functions()
+self.check_partitioning_functions(DataFrameWriterV2)
 
 
 if __name__ == "__main__":
diff --git a/python/pyspark/sql/tests/test_readwriter.py 
b/python/pyspark/sql/tests/test_readwriter.py
index 23b607fe98e..9cd3e613667 100644
--- a/python/pyspark/sql/tests/test_readwriter.py
+++ b/python/pyspark/sql/tests/test_readwriter.py
@@ -181,17 +181,23 @@ class ReadwriterTestsMixin:
 
 class ReadwriterV2TestsMixin:
 def test_api(self):
+self.check_api(DataFrameWriterV2)
+
+def check_api(self, tpe):
 df = self.df
 writer = df.writeTo("testcat.t")
-self.assertIsInstance(writer, DataFrameWriterV2)
-self.assertIsInstance(writer.option("property", "value"), 
DataFrameWriterV2)
-self.assertIsInstance(writer.options(property="value"), 
DataFrameWriterV2)
-self.assertIsInstance(writer.using("source"), DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy("id"), DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(col("id")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.tableProperty("foo", "bar"), 
DataFrameWriterV2)
+self.assertIsInstance(writer, tpe)
+self.assertIsInstance(writer.option("property", "value"), tpe)
+self.assertIsInstance(writer.options(property="value"), tpe)
+self.assertIsInstance(writer.using("source"), tpe)
+self.assertIsInstance(writer.partitionedBy("id"), tpe)
+self.assertIsInstance(writer.partitionedBy(col("id")), tpe)
+self.assertIsInstance(writer.tableProperty("foo", "bar"), tpe)
 
 def test_partitioning_functions(self):
+self.check_partitioning_functions(DataFrameWriterV2)
+
+def check_partitioning_functions(self, tpe):
 import datetime
 from pyspark.sql.functions import years, months, days, hours, bucket
 
@@ -201,15 +207,13 @@ class ReadwriterV2TestsMixin:
 
 writer = df.writeTo("testcat.t")
 
-self.assertIsInstance(writer.partitionedBy(years("ts")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(months("ts")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(days("ts")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(hours("ts")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(bucket(11, "id")), 
DataFrameWriterV2)
-

[spark] branch master updated: [SPARK-42002][CONNECT][PYTHON][FOLLOWUP] Enable tests in ReadwriterV2ParityTests

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

ruifengz 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 961d5bd9097 [SPARK-42002][CONNECT][PYTHON][FOLLOWUP] Enable tests in 
ReadwriterV2ParityTests
961d5bd9097 is described below

commit 961d5bd909744fea24e2391cd1a7aea3c96c418d
Author: Takuya UESHIN 
AuthorDate: Fri Feb 17 09:26:12 2023 +0800

[SPARK-42002][CONNECT][PYTHON][FOLLOWUP] Enable tests in 
ReadwriterV2ParityTests

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

Enables tests in `ReadwriterV2ParityTests`.

### Why are the changes needed?

Now that `DataFrameWriterV2` for Spark Connect is implemented, we can 
enable tests in `ReadwriterV2ParityTests`.

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

No.

### How was this patch tested?

Enabled tests.

Closes #40060 from ueshin/issues/SPARK-42002/tests.

Authored-by: Takuya UESHIN 
Signed-off-by: Ruifeng Zheng 
---
 .../sql/tests/connect/test_parity_readwriter.py|  9 ++
 python/pyspark/sql/tests/test_readwriter.py| 36 --
 2 files changed, 23 insertions(+), 22 deletions(-)

diff --git a/python/pyspark/sql/tests/connect/test_parity_readwriter.py 
b/python/pyspark/sql/tests/connect/test_parity_readwriter.py
index 61f542385ec..bf77043ef38 100644
--- a/python/pyspark/sql/tests/connect/test_parity_readwriter.py
+++ b/python/pyspark/sql/tests/connect/test_parity_readwriter.py
@@ -16,6 +16,7 @@
 #
 import unittest
 
+from pyspark.sql.connect.readwriter import DataFrameWriterV2
 from pyspark.sql.tests.test_readwriter import ReadwriterTestsMixin, 
ReadwriterV2TestsMixin
 from pyspark.testing.connectutils import ReusedConnectTestCase
 
@@ -33,15 +34,11 @@ class ReadwriterParityTests(ReadwriterTestsMixin, 
ReusedConnectTestCase):
 
 
 class ReadwriterV2ParityTests(ReadwriterV2TestsMixin, ReusedConnectTestCase):
-# TODO(SPARK-42002): Implement writeTo()
-@unittest.skip("Fails in Spark Connect, should enable.")
 def test_api(self):
-super().test_api()
+self.check_api(DataFrameWriterV2)
 
-# TODO(SPARK-42002): Implement writeTo()
-@unittest.skip("Fails in Spark Connect, should enable.")
 def test_partitioning_functions(self):
-super().test_partitioning_functions()
+self.check_partitioning_functions(DataFrameWriterV2)
 
 
 if __name__ == "__main__":
diff --git a/python/pyspark/sql/tests/test_readwriter.py 
b/python/pyspark/sql/tests/test_readwriter.py
index 23b607fe98e..9cd3e613667 100644
--- a/python/pyspark/sql/tests/test_readwriter.py
+++ b/python/pyspark/sql/tests/test_readwriter.py
@@ -181,17 +181,23 @@ class ReadwriterTestsMixin:
 
 class ReadwriterV2TestsMixin:
 def test_api(self):
+self.check_api(DataFrameWriterV2)
+
+def check_api(self, tpe):
 df = self.df
 writer = df.writeTo("testcat.t")
-self.assertIsInstance(writer, DataFrameWriterV2)
-self.assertIsInstance(writer.option("property", "value"), 
DataFrameWriterV2)
-self.assertIsInstance(writer.options(property="value"), 
DataFrameWriterV2)
-self.assertIsInstance(writer.using("source"), DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy("id"), DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(col("id")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.tableProperty("foo", "bar"), 
DataFrameWriterV2)
+self.assertIsInstance(writer, tpe)
+self.assertIsInstance(writer.option("property", "value"), tpe)
+self.assertIsInstance(writer.options(property="value"), tpe)
+self.assertIsInstance(writer.using("source"), tpe)
+self.assertIsInstance(writer.partitionedBy("id"), tpe)
+self.assertIsInstance(writer.partitionedBy(col("id")), tpe)
+self.assertIsInstance(writer.tableProperty("foo", "bar"), tpe)
 
 def test_partitioning_functions(self):
+self.check_partitioning_functions(DataFrameWriterV2)
+
+def check_partitioning_functions(self, tpe):
 import datetime
 from pyspark.sql.functions import years, months, days, hours, bucket
 
@@ -201,15 +207,13 @@ class ReadwriterV2TestsMixin:
 
 writer = df.writeTo("testcat.t")
 
-self.assertIsInstance(writer.partitionedBy(years("ts")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(months("ts")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(days("ts")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(hours("ts")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(bucket(11, "id")), 
DataFrameWriterV2)
-self.assertIsInstance(writer.partitionedBy(bucket(11, col("id"))), 
DataFrameWriterV2)
-self.assertIsInstance(
-

[spark] branch branch-3.4 updated: [SPARK-39904][SQL][FOLLOW-UP] Rename CSV option `prefersDate` as `preferDate`

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 0efbfd57401 [SPARK-39904][SQL][FOLLOW-UP] Rename CSV option 
`prefersDate` as `preferDate`
0efbfd57401 is described below

commit 0efbfd574011dc19cac74ee25043b0a1d57e8497
Author: Gengliang Wang 
AuthorDate: Thu Feb 16 17:23:15 2023 -0800

[SPARK-39904][SQL][FOLLOW-UP] Rename CSV option `prefersDate` as 
`preferDate`

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

Rename the CSV data source option `prefersDate` as `preferDate`.

### Why are the changes needed?

All the CSV data source options doesn't have a `s` on the verb in the 
naming. For example, `inferSchema`, `ignoreLeadingWhiteSpace` and 
`ignoreTrailingWhiteSpace`.
The renaming makes the naming consistent.
Also, the title of JIRA https://issues.apache.org/jira/browse/SPARK-39904 
uses `preferDate` as well.

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

No, the data source option is not released yet.

### How was this patch tested?

Existing UT

Closes #40043 from gengliangwang/renameCSVOption.

Authored-by: Gengliang Wang 
Signed-off-by: Gengliang Wang 
(cherry picked from commit 6ead12e4ac08cef6c1346df3d380e85e5937a842)
Signed-off-by: Gengliang Wang 
---
 docs/sql-data-sources-csv.md   |  2 +-
 .../org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala |  2 +-
 .../scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala   | 10 +-
 .../apache/spark/sql/execution/datasources/csv/CSVSuite.scala  |  8 
 4 files changed, 11 insertions(+), 11 deletions(-)

diff --git a/docs/sql-data-sources-csv.md b/docs/sql-data-sources-csv.md
index 42b117bea46..be53f0301c0 100644
--- a/docs/sql-data-sources-csv.md
+++ b/docs/sql-data-sources-csv.md
@@ -109,7 +109,7 @@ Data source options of CSV can be set via:
 read
   
   
-prefersDate
+preferDate
 true
 During schema inference (inferSchema), attempts to infer 
string columns that contain dates as Date if the values satisfy 
the dateFormat option or default date format. For columns that 
contain a mixture of dates and timestamps, try inferring them as 
TimestampType if timestamp format not specified, otherwise infer 
them as StringType.
 read
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
index bdfa4ac3f0f..51586a0065e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala
@@ -183,7 +183,7 @@ class CSVInferSchema(val options: CSVOptions) extends 
Serializable {
   private def tryParseDouble(field: String): DataType = {
 if ((allCatch opt field.toDouble).isDefined || isInfOrNan(field)) {
   DoubleType
-} else if (options.prefersDate) {
+} else if (options.preferDate) {
   tryParseDate(field)
 } else {
   tryParseTimestampNTZ(field)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala
index a66070aa853..1a9de5bc35e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala
@@ -159,19 +159,19 @@ class CSVOptions(
* Not compatible with legacyTimeParserPolicy == LEGACY since legacy date 
parser will accept
* extra trailing characters. Thus, disabled when legacyTimeParserPolicy == 
LEGACY
*/
-  val prefersDate = {
+  val preferDate = {
 if (SQLConf.get.legacyTimeParserPolicy == LegacyBehaviorPolicy.LEGACY) {
   false
 } else {
-  getBool(PREFERS_DATE, true)
+  getBool(PREFER_DATE, true)
 }
   }
 
   val dateFormatOption: Option[String] = parameters.get(DATE_FORMAT)
-  // Provide a default value for dateFormatInRead when prefersDate. This 
ensures that the
+  // Provide a default value for dateFormatInRead when preferDate. This 
ensures that the
   // Iso8601DateFormatter (with strict date parsing) is used for date inference
   val dateFormatInRead: Option[String] =
-if (prefersDate) {
+if (preferDate) {
   Option(dateFormatOption.getOrElse(DateFormatter.defaultPattern))
 } else {
   dateFormatOption
@@ -335,7 +335,7 @@ object CSVOptions extends DataSourceOptions {
   val INFER_SCHEMA = newOption("inferSchema")
   val IGNORE_LEADING_WHITESPACE = newOption("ignoreLeadingWhiteSpace")
   val IGNORE_TRAILING_WHITESPACE = newOption("ignoreTrailingWhiteSpace")
-  val PREFERS_DATE = 

[spark] branch master updated (64aa84b21c7 -> 6ead12e4ac0)

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

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


from 64aa84b21c7 [SPARK-42465][CONNECT] ProtoToPlanTestSuite should use 
analyzed plans instead of parsed plans
 add 6ead12e4ac0 [SPARK-39904][SQL][FOLLOW-UP] Rename CSV option 
`prefersDate` as `preferDate`

No new revisions were added by this update.

Summary of changes:
 docs/sql-data-sources-csv.md   |  2 +-
 .../org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala |  2 +-
 .../scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala   | 10 +-
 .../apache/spark/sql/execution/datasources/csv/CSVSuite.scala  |  8 
 4 files changed, 11 insertions(+), 11 deletions(-)


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



[GitHub] [spark-website] srowen commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


srowen commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433869775

   Although looks like you are hung up just fetching. Fetch asf-site first? See 
if that works


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark-website] dongjoon-hyun commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


dongjoon-hyun commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433869742

   I used the script to merge this~
   
   ```
   $ ./merge_pr.py
   git rev-parse --abbrev-ref HEAD
   Which pull request would you like to merge? (e.g. 34): 435
   
   === Pull Request #435 ===
   titleAdd docs for Apache Spark 3.3.2
   source   viirya/add_doc_3.3.2
   target   asf-site
   url  https://api.github.com/repos/apache/spark-website/pulls/435
   
   Proceed with merging pull request #435? (y/n): y
   git fetch apache pull/435/head:PR_TOOL_MERGE_PR_435
   remote: Enumerating objects: 8903, done.
   remote: Counting objects: 100% (8903/8903), done.
   remote: Compressing objects: 100% (2274/2274), done.
   remote: Total 8899 (delta 6645), reused 8828 (delta 6609), pack-reused 0
   Receiving objects: 100% (8899/8899), 36.03 MiB | 16.96 MiB/s, done.
   Resolving deltas: 100% (6645/6645), completed with 3 local objects.
   From github.com:apache/spark-website
* [new ref] refs/pull/435/head -> PR_TOOL_MERGE_PR_435
   git fetch apache asf-site:PR_TOOL_MERGE_PR_435_ASF-SITE
   From github.com:apache/spark-website
* [new branch]  asf-site   -> PR_TOOL_MERGE_PR_435_ASF-SITE
   git checkout PR_TOOL_MERGE_PR_435_ASF-SITE
   Switched to branch 'PR_TOOL_MERGE_PR_435_ASF-SITE'
   ['git', 'merge', 'PR_TOOL_MERGE_PR_435', '--squash']
   Updating files: 100% (8590/8590), done.
   ['git', 'log', 'HEAD..PR_TOOL_MERGE_PR_435', '--pretty=format:%an <%ae>']
   Enter primary author in the format of "name " [Liang-Chi Hsieh 
]:
   ['git', 'log', 'HEAD..PR_TOOL_MERGE_PR_435', '--pretty=format:%h [%an] %s']
   ['git', 'commit', '--author="Liang-Chi Hsieh "', '-m', 
'Add docs for Apache Spark 3.3.2', '-m', 'Adds docs from 
https://dist.apache.org/repos/dist/dev/spark/v3.3.2-rc1-docs/', '-m', 'Author: 
Liang-Chi Hsieh ', '-m', 'Closes #435 from 
viirya/add_doc_3.3.2.']
   
   Merge complete (local ref PR_TOOL_MERGE_PR_435_ASF-SITE). Push to apache? 
(y/n): y
   git push apache PR_TOOL_MERGE_PR_435_ASF-SITE:asf-site
   Enumerating objects: 8903, done.
   Counting objects: 100% (8903/8903), done.
   Delta compression using up to 10 threads
   Compressing objects: 100% (2238/2238), done.
   Writing objects: 100% (8899/8899), 36.03 MiB | 6.38 MiB/s, done.
   Total 8899 (delta 6645), reused 8898 (delta 6645), pack-reused 0
   remote: Resolving deltas: 100% (6645/6645), completed with 3 local objects.
   remote: Checking connectivity: 6618, done.
   To github.com:apache/spark-website.git
  551848fe0..d11ffbdb6  PR_TOOL_MERGE_PR_435_ASF-SITE -> asf-site
   git rev-parse PR_TOOL_MERGE_PR_435_ASF-SITE
   Restoring head pointer to asf-site
   git checkout asf-site
   Switched to branch 'asf-site'
   git branch
   Deleting local branch PR_TOOL_MERGE_PR_435
   git branch -D PR_TOOL_MERGE_PR_435
   Deleting local branch PR_TOOL_MERGE_PR_435_ASF-SITE
   git branch -D PR_TOOL_MERGE_PR_435_ASF-SITE
   Pull request #435 merged!
   Merge hash: d11ffbdb
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark-website] srowen commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


srowen commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433869373

   Wow no. I suppose you can use the merge button if you have to 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark-website] dongjoon-hyun closed pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


dongjoon-hyun closed pull request #435: Add docs for Apache Spark 3.3.2
URL: https://github.com/apache/spark-website/pull/435


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark-website] viirya commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


viirya commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433870394

   Thanks @dongjoon-hyun !
   
   @srowen I did a separate fetch but got same timeout error.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
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.4.0-rc1

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

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

commit 1845fa200cbf4283e8be5fdd240821b7a13b00b7
Author: Xinrong Meng 
AuthorDate: Thu Feb 16 23:07:56 2023 +

Preparing Spark release v3.4.0-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 +-
 connector/avro/pom.xml | 2 +-
 connector/connect/client/jvm/pom.xml   | 2 +-
 connector/connect/common/pom.xml   | 2 +-
 connector/connect/server/pom.xml   | 2 +-
 connector/docker-integration-tests/pom.xml | 2 +-
 connector/kafka-0-10-assembly/pom.xml  | 2 +-
 connector/kafka-0-10-sql/pom.xml   | 2 +-
 connector/kafka-0-10-token-provider/pom.xml| 2 +-
 connector/kafka-0-10/pom.xml   | 2 +-
 connector/kinesis-asl-assembly/pom.xml | 2 +-
 connector/kinesis-asl/pom.xml  | 2 +-
 connector/protobuf/pom.xml | 2 +-
 connector/spark-ganglia-lgpl/pom.xml   | 2 +-
 core/pom.xml   | 2 +-
 docs/_config.yml   | 4 ++--
 examples/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 +-
 42 files changed, 43 insertions(+), 43 deletions(-)

diff --git a/assembly/pom.xml b/assembly/pom.xml
index f37edcd7e49..58dd9ef46e0 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,7 +21,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0-SNAPSHOT
+3.4.0
 ../pom.xml
   
 
diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml
index d19883549d0..95ea15552da 100644
--- a/common/kvstore/pom.xml
+++ b/common/kvstore/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml
index ff2cc71c2c9..e4d98471bf9 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.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml
index de4730c5b71..7a6d5aedf65 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.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml
index a77732bb8b8..1c421754083 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.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml
index f4e442b3ba7..2ee25ebfffc 100644
--- a/common/sketch/pom.xml
+++ b/common/sketch/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0-SNAPSHOT
+3.4.0
 ../../pom.xml
   
 
diff --git a/common/tags/pom.xml b/common/tags/pom.xml
index 216812152ad..5c31e6eb365 100644
--- a/common/tags/pom.xml
+++ b/common/tags/pom.xml
@@ -22,7 +22,7 @@
   
 org.apache.spark
 spark-parent_2.12
-3.4.0-SNAPSHOT
+

[spark] tag v3.4.0-rc1 created (now 1845fa200cb)

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

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


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

 new 1845fa200cb Preparing Spark release v3.4.0-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



[GitHub] [spark-website] viirya commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


viirya commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433822446

   Hm, have you seen this:
   
   ```
   ./merge_pr.py
   git rev-parse --abbrev-ref HEAD
   Which pull request would you like to merge? (e.g. 34): 435
   
   === Pull Request #435 ===
   title   Add docs for Apache Spark 3.3.2
   source  viirya/add_doc_3.3.2
   target  asf-site
   url https://api.github.com/repos/apache/spark-website/pulls/435
   
   Proceed with merging pull request #435? (y/n): y
   git fetch apache-github pull/435/head:PR_TOOL_MERGE_PR_435
   fatal: unable to connect to github.com:
   github.com[0: 192.30.255.113]: errno=Operation timed out
   ```
   
   Seems too many files to fetch so it timed out.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[spark] branch branch-3.4 updated: [SPARK-42465][CONNECT] ProtoToPlanTestSuite should use analyzed plans instead of parsed plans

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new cb1fb7a2a47 [SPARK-42465][CONNECT] ProtoToPlanTestSuite should use 
analyzed plans instead of parsed plans
cb1fb7a2a47 is described below

commit cb1fb7a2a472d5f25cd77f0f1dcdc7055243982d
Author: Herman van Hovell 
AuthorDate: Thu Feb 16 18:33:10 2023 -0400

[SPARK-42465][CONNECT] ProtoToPlanTestSuite should use analyzed plans 
instead of parsed plans

### What changes were proposed in this pull request?
This makes `ProtoToPlanTestSuite` use analyzed plans instead of parsed 
plans.

### Why are the changes needed?
This is to increase the fidelity of the `ProtoToPlanTestSuite`, especially 
since we are going to be adding functions. Functions are special because the 
spark connect planner leaves them unresolved, the actual binding only happens 
in the analyzer. Without running the analyzer we would not know if the bindings 
are correct.

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

### How was this patch tested?
It is a test change.

Closes #40056 from hvanhovell/SPARK-42465.

Authored-by: Herman van Hovell 
Signed-off-by: Herman van Hovell 
(cherry picked from commit 64aa84b21c7664205a060933d8d9c3067da8218b)
Signed-off-by: Herman van Hovell 
---
 .../apache/spark/sql/PlanGenerationTestSuite.scala |  16 -
 .../explain-results/alias_string.explain   |   3 +-
 .../explain-results/alias_symbol.explain   |   3 +-
 .../query-tests/explain-results/apply.explain  |   4 +--
 .../query-tests/explain-results/as_string.explain  |   3 +-
 .../query-tests/explain-results/as_symbol.explain  |   3 +-
 .../query-tests/explain-results/coalesce.explain   |   2 +-
 .../query-tests/explain-results/col.explain|   4 +--
 .../query-tests/explain-results/colRegex.explain   |   4 +--
 .../query-tests/explain-results/column_add.explain |   4 +--
 .../explain-results/column_alias.explain   |   4 +--
 .../query-tests/explain-results/column_and.explain |   4 +--
 .../explain-results/column_apply.explain   |   4 +--
 .../explain-results/column_as_multi.explain|   5 +--
 .../column_as_with_metadata.explain|   4 +--
 .../query-tests/explain-results/column_asc.explain |   4 +--
 .../explain-results/column_asc_nulls_first.explain |   4 +--
 .../explain-results/column_asc_nulls_last.explain  |   4 +--
 .../explain-results/column_between.explain |   4 +--
 .../explain-results/column_bitwiseAND.explain  |   4 +--
 .../explain-results/column_bitwiseOR.explain   |   4 +--
 .../explain-results/column_bitwiseXOR.explain  |   4 +--
 .../explain-results/column_cast.explain|   4 +--
 .../explain-results/column_contains.explain|   4 +--
 .../explain-results/column_desc.explain|   4 +--
 .../column_desc_nulls_first.explain|   4 +--
 .../explain-results/column_desc_nulls_last.explain |   4 +--
 .../explain-results/column_divide.explain  |   4 +--
 .../explain-results/column_dropFields.explain  |   4 +--
 .../explain-results/column_endsWith.explain|   4 +--
 .../explain-results/column_eqNullSafe.explain  |   4 +--
 .../explain-results/column_equals.explain  |   4 +--
 .../query-tests/explain-results/column_geq.explain |   4 +--
 .../explain-results/column_getField.explain|   4 +--
 .../explain-results/column_getItem.explain |   4 +--
 .../query-tests/explain-results/column_gt.explain  |   4 +--
 .../explain-results/column_ilike.explain   |   4 +--
 .../explain-results/column_isNaN.explain   |   4 +--
 .../explain-results/column_isNotNull.explain   |   4 +--
 .../explain-results/column_isNull.explain  |   4 +--
 .../explain-results/column_isin.explain|   4 +--
 .../query-tests/explain-results/column_leq.explain |   4 +--
 .../explain-results/column_like.explain|   4 +--
 .../query-tests/explain-results/column_lt.explain  |   4 +--
 .../explain-results/column_modulo.explain  |   4 +--
 .../explain-results/column_multiply.explain|   4 +--
 .../query-tests/explain-results/column_not.explain |   4 +--
 .../explain-results/column_not_equals.explain  |   4 +--
 .../query-tests/explain-results/column_or.explain  |   4 +--
 .../explain-results/column_rlike.explain   |   4 +--
 .../explain-results/column_star.explain|   4 +--
 .../column_star_with_target.explain|   4 +--
 .../explain-results/column_startsWith.explain  |   4 +--
 .../explain-results/column_substr.explain  |   4 +--
 .../explain-results/column_subtract.explain|   4 +--
 .../explain-results/column_unary_minus.explain |   4 +--
 

[spark] branch master updated: [SPARK-42465][CONNECT] ProtoToPlanTestSuite should use analyzed plans instead of parsed plans

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

hvanhovell 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 64aa84b21c7 [SPARK-42465][CONNECT] ProtoToPlanTestSuite should use 
analyzed plans instead of parsed plans
64aa84b21c7 is described below

commit 64aa84b21c7664205a060933d8d9c3067da8218b
Author: Herman van Hovell 
AuthorDate: Thu Feb 16 18:33:10 2023 -0400

[SPARK-42465][CONNECT] ProtoToPlanTestSuite should use analyzed plans 
instead of parsed plans

### What changes were proposed in this pull request?
This makes `ProtoToPlanTestSuite` use analyzed plans instead of parsed 
plans.

### Why are the changes needed?
This is to increase the fidelity of the `ProtoToPlanTestSuite`, especially 
since we are going to be adding functions. Functions are special because the 
spark connect planner leaves them unresolved, the actual binding only happens 
in the analyzer. Without running the analyzer we would not know if the bindings 
are correct.

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

### How was this patch tested?
It is a test change.

Closes #40056 from hvanhovell/SPARK-42465.

Authored-by: Herman van Hovell 
Signed-off-by: Herman van Hovell 
---
 .../apache/spark/sql/PlanGenerationTestSuite.scala |  16 -
 .../explain-results/alias_string.explain   |   3 +-
 .../explain-results/alias_symbol.explain   |   3 +-
 .../query-tests/explain-results/apply.explain  |   4 +--
 .../query-tests/explain-results/as_string.explain  |   3 +-
 .../query-tests/explain-results/as_symbol.explain  |   3 +-
 .../query-tests/explain-results/coalesce.explain   |   2 +-
 .../query-tests/explain-results/col.explain|   4 +--
 .../query-tests/explain-results/colRegex.explain   |   4 +--
 .../query-tests/explain-results/column_add.explain |   4 +--
 .../explain-results/column_alias.explain   |   4 +--
 .../query-tests/explain-results/column_and.explain |   4 +--
 .../explain-results/column_apply.explain   |   4 +--
 .../explain-results/column_as_multi.explain|   5 +--
 .../column_as_with_metadata.explain|   4 +--
 .../query-tests/explain-results/column_asc.explain |   4 +--
 .../explain-results/column_asc_nulls_first.explain |   4 +--
 .../explain-results/column_asc_nulls_last.explain  |   4 +--
 .../explain-results/column_between.explain |   4 +--
 .../explain-results/column_bitwiseAND.explain  |   4 +--
 .../explain-results/column_bitwiseOR.explain   |   4 +--
 .../explain-results/column_bitwiseXOR.explain  |   4 +--
 .../explain-results/column_cast.explain|   4 +--
 .../explain-results/column_contains.explain|   4 +--
 .../explain-results/column_desc.explain|   4 +--
 .../column_desc_nulls_first.explain|   4 +--
 .../explain-results/column_desc_nulls_last.explain |   4 +--
 .../explain-results/column_divide.explain  |   4 +--
 .../explain-results/column_dropFields.explain  |   4 +--
 .../explain-results/column_endsWith.explain|   4 +--
 .../explain-results/column_eqNullSafe.explain  |   4 +--
 .../explain-results/column_equals.explain  |   4 +--
 .../query-tests/explain-results/column_geq.explain |   4 +--
 .../explain-results/column_getField.explain|   4 +--
 .../explain-results/column_getItem.explain |   4 +--
 .../query-tests/explain-results/column_gt.explain  |   4 +--
 .../explain-results/column_ilike.explain   |   4 +--
 .../explain-results/column_isNaN.explain   |   4 +--
 .../explain-results/column_isNotNull.explain   |   4 +--
 .../explain-results/column_isNull.explain  |   4 +--
 .../explain-results/column_isin.explain|   4 +--
 .../query-tests/explain-results/column_leq.explain |   4 +--
 .../explain-results/column_like.explain|   4 +--
 .../query-tests/explain-results/column_lt.explain  |   4 +--
 .../explain-results/column_modulo.explain  |   4 +--
 .../explain-results/column_multiply.explain|   4 +--
 .../query-tests/explain-results/column_not.explain |   4 +--
 .../explain-results/column_not_equals.explain  |   4 +--
 .../query-tests/explain-results/column_or.explain  |   4 +--
 .../explain-results/column_rlike.explain   |   4 +--
 .../explain-results/column_star.explain|   4 +--
 .../column_star_with_target.explain|   4 +--
 .../explain-results/column_startsWith.explain  |   4 +--
 .../explain-results/column_substr.explain  |   4 +--
 .../explain-results/column_subtract.explain|   4 +--
 .../explain-results/column_unary_minus.explain |   4 +--
 .../explain-results/column_when_otherwise.explain  |   4 +--
 .../explain-results/column_withField.explain   |   4 +--
 

[GitHub] [spark-website] srowen commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


srowen commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433714106

   Use the script 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark-website] viirya commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


viirya commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433699384

   I forgot how do we merge pr into spark-website. Do we use `merge_pr.py` or 
just click "Squash and merge" button?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark-website] dongjoon-hyun commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


dongjoon-hyun commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433648243

   +1, LGTM!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[GitHub] [spark-website] viirya commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


viirya commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433614031

   cc @dongjoon-hyun 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[spark] branch branch-3.4 updated: [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE`

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 7e2642db062 [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` 
into `UNSUPPORTED_DATATYPE`
7e2642db062 is described below

commit 7e2642db062cc45c44cbf549d6431bd72915fa17
Author: itholic 
AuthorDate: Thu Feb 16 22:04:17 2023 +0300

[SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into 
`UNSUPPORTED_DATATYPE`

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

This PR proposes to integrate `_LEGACY_ERROR_TEMP_2099` into 
`UNSUPPORTED_DATATYPE`.

And also introduce new error class `UNSUPPORTED_ARROWTYPE`.

### Why are the changes needed?

We should assign proper name for LEGACY errors.

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

No.

### How was this patch tested?

Updated UT.

Closes #39979 from itholic/LEGACY_2099.

Authored-by: itholic 
Signed-off-by: Max Gekk 
(cherry picked from commit 9855b137032bf9504dff96eb5bb9951accacac0f)
Signed-off-by: Max Gekk 
---
 core/src/main/resources/error/error-classes.json| 11 ++-
 .../org/apache/spark/sql/errors/QueryExecutionErrors.scala  | 13 ++---
 .../main/scala/org/apache/spark/sql/util/ArrowUtils.scala   |  4 ++--
 .../scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala   |  9 -
 .../scala/org/apache/spark/sql/execution/Columnar.scala |  2 +-
 .../org/apache/spark/sql/execution/arrow/ArrowWriter.scala  |  2 +-
 .../spark/sql/execution/arrow/ArrowConvertersSuite.scala| 10 ++
 7 files changed, 34 insertions(+), 17 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 770223625cf..a0970550d72 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1567,6 +1567,12 @@
 ],
 "sqlState" : "42703"
   },
+  "UNSUPPORTED_ARROWTYPE" : {
+"message" : [
+  "Unsupported arrow type ."
+],
+"sqlState" : "0A000"
+  },
   "UNSUPPORTED_DATATYPE" : {
 "message" : [
   "Unsupported data type ."
@@ -4094,11 +4100,6 @@
   "Could not compare cost with ."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2099" : {
-"message" : [
-  "Unsupported data type: ."
-]
-  },
   "_LEGACY_ERROR_TEMP_2100" : {
 "message" : [
   "not support type: ."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index 2bafa2e2c03..17c5b2f4f10 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -25,6 +25,7 @@ import java.time.temporal.ChronoField
 import java.util.concurrent.TimeoutException
 
 import com.fasterxml.jackson.core.{JsonParser, JsonToken}
+import org.apache.arrow.vector.types.pojo.ArrowType
 import org.apache.hadoop.fs.{FileAlreadyExistsException, FileStatus, Path}
 import org.apache.hadoop.fs.permission.FsPermission
 import org.codehaus.commons.compiler.{CompileException, 
InternalCompilerException}
@@ -1124,10 +1125,16 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase {
   messageParameters = Map("cost" -> cost))
   }
 
-  def unsupportedDataTypeError(dt: String): SparkUnsupportedOperationException 
= {
+  def unsupportedArrowTypeError(typeName: ArrowType): 
SparkUnsupportedOperationException = {
 new SparkUnsupportedOperationException(
-  errorClass = "_LEGACY_ERROR_TEMP_2099",
-  messageParameters = Map("dt" -> dt))
+  errorClass = "UNSUPPORTED_ARROWTYPE",
+  messageParameters = Map("typeName" -> typeName.toString))
+  }
+
+  def unsupportedDataTypeError(typeName: DataType): 
SparkUnsupportedOperationException = {
+new SparkUnsupportedOperationException(
+  errorClass = "UNSUPPORTED_DATATYPE",
+  messageParameters = Map("typeName" -> toSQLType(typeName)))
   }
 
   def notSupportTypeError(dataType: DataType): Throwable = {
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
index e854eba0383..6c6635bac57 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
@@ -56,7 +56,7 @@ private[sql] object ArrowUtils {
 case _: YearMonthIntervalType => new 
ArrowType.Interval(IntervalUnit.YEAR_MONTH)
 case _: DayTimeIntervalType => new ArrowType.Duration(TimeUnit.MICROSECOND)
 case _ =>
-  throw 

[spark] branch master updated: [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into `UNSUPPORTED_DATATYPE`

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

maxgekk 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 9855b137032 [SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` 
into `UNSUPPORTED_DATATYPE`
9855b137032 is described below

commit 9855b137032bf9504dff96eb5bb9951accacac0f
Author: itholic 
AuthorDate: Thu Feb 16 22:04:17 2023 +0300

[SPARK-42326][SQL] Integrate `_LEGACY_ERROR_TEMP_2099` into 
`UNSUPPORTED_DATATYPE`

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

This PR proposes to integrate `_LEGACY_ERROR_TEMP_2099` into 
`UNSUPPORTED_DATATYPE`.

And also introduce new error class `UNSUPPORTED_ARROWTYPE`.

### Why are the changes needed?

We should assign proper name for LEGACY errors.

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

No.

### How was this patch tested?

Updated UT.

Closes #39979 from itholic/LEGACY_2099.

Authored-by: itholic 
Signed-off-by: Max Gekk 
---
 core/src/main/resources/error/error-classes.json| 11 ++-
 .../org/apache/spark/sql/errors/QueryExecutionErrors.scala  | 13 ++---
 .../main/scala/org/apache/spark/sql/util/ArrowUtils.scala   |  4 ++--
 .../scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala   |  9 -
 .../scala/org/apache/spark/sql/execution/Columnar.scala |  2 +-
 .../org/apache/spark/sql/execution/arrow/ArrowWriter.scala  |  2 +-
 .../spark/sql/execution/arrow/ArrowConvertersSuite.scala| 10 ++
 7 files changed, 34 insertions(+), 17 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json 
b/core/src/main/resources/error/error-classes.json
index 01fd43eec7f..6d9aa59e57f 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1567,6 +1567,12 @@
 ],
 "sqlState" : "42703"
   },
+  "UNSUPPORTED_ARROWTYPE" : {
+"message" : [
+  "Unsupported arrow type ."
+],
+"sqlState" : "0A000"
+  },
   "UNSUPPORTED_DATATYPE" : {
 "message" : [
   "Unsupported data type ."
@@ -4094,11 +4100,6 @@
   "Could not compare cost with ."
 ]
   },
-  "_LEGACY_ERROR_TEMP_2099" : {
-"message" : [
-  "Unsupported data type: ."
-]
-  },
   "_LEGACY_ERROR_TEMP_2100" : {
 "message" : [
   "not support type: ."
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index 2bafa2e2c03..17c5b2f4f10 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -25,6 +25,7 @@ import java.time.temporal.ChronoField
 import java.util.concurrent.TimeoutException
 
 import com.fasterxml.jackson.core.{JsonParser, JsonToken}
+import org.apache.arrow.vector.types.pojo.ArrowType
 import org.apache.hadoop.fs.{FileAlreadyExistsException, FileStatus, Path}
 import org.apache.hadoop.fs.permission.FsPermission
 import org.codehaus.commons.compiler.{CompileException, 
InternalCompilerException}
@@ -1124,10 +1125,16 @@ private[sql] object QueryExecutionErrors extends 
QueryErrorsBase {
   messageParameters = Map("cost" -> cost))
   }
 
-  def unsupportedDataTypeError(dt: String): SparkUnsupportedOperationException 
= {
+  def unsupportedArrowTypeError(typeName: ArrowType): 
SparkUnsupportedOperationException = {
 new SparkUnsupportedOperationException(
-  errorClass = "_LEGACY_ERROR_TEMP_2099",
-  messageParameters = Map("dt" -> dt))
+  errorClass = "UNSUPPORTED_ARROWTYPE",
+  messageParameters = Map("typeName" -> typeName.toString))
+  }
+
+  def unsupportedDataTypeError(typeName: DataType): 
SparkUnsupportedOperationException = {
+new SparkUnsupportedOperationException(
+  errorClass = "UNSUPPORTED_DATATYPE",
+  messageParameters = Map("typeName" -> toSQLType(typeName)))
   }
 
   def notSupportTypeError(dataType: DataType): Throwable = {
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
index e854eba0383..6c6635bac57 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
@@ -56,7 +56,7 @@ private[sql] object ArrowUtils {
 case _: YearMonthIntervalType => new 
ArrowType.Interval(IntervalUnit.YEAR_MONTH)
 case _: DayTimeIntervalType => new ArrowType.Duration(TimeUnit.MICROSECOND)
 case _ =>
-  throw QueryExecutionErrors.unsupportedDataTypeError(dt.catalogString)
+  throw QueryExecutionErrors.unsupportedDataTypeError(dt)
   }
 
   def fromArrowType(dt: 

[spark] branch branch-3.4 updated: [SPARK-42464][CONNECT] Fix ProtoToPlanTestSuite for Scala 2.13

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 1c70f5f2d73 [SPARK-42464][CONNECT] Fix ProtoToPlanTestSuite for Scala 
2.13
1c70f5f2d73 is described below

commit 1c70f5f2d73331360b53993d0f2e73e7a7c8da4a
Author: Herman van Hovell 
AuthorDate: Thu Feb 16 14:47:05 2023 -0400

[SPARK-42464][CONNECT] Fix ProtoToPlanTestSuite for Scala 2.13

### What changes were proposed in this pull request?
The `ProtoToPlanTestSuite` were broken for Scala 2.13 . This was caused by 
the following two problems:
- Explain output between 2.12 and 2.13 is not stable because we render 
collection implementations as well. For this I changed the rendering of the 
offending classes to be version agnostic.
- UDF code had deserialization issues. This was always the risk. I have 
removed those tests, we will work on improving UDF coverage in a follow-up.

### Why are the changes needed?
We want to test Scala 2.13 as well.

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

### How was this patch tested?
Test only change.

Closes #40055 from hvanhovell/SPARK-42464.

Authored-by: Herman van Hovell 
Signed-off-by: Herman van Hovell 
(cherry picked from commit 229807efd06562fc07fce9a257062d8a66068761)
Signed-off-by: Herman van Hovell 
---
 .../apache/spark/sql/PlanGenerationTestSuite.scala |  21 -
 .../explain-results/column_as_multi.explain|   2 +-
 .../column_star_with_target.explain|   2 +-
 .../explain-results/function_udf.explain   |   2 -
 .../explain-results/function_udf_2.12.explain  |   2 -
 .../explain-results/function_udf_2.13.explain  |   2 -
 .../join_inner_using_multiple_col_array.explain|   2 +-
 .../join_inner_using_multiple_col_seq.explain  |   2 +-
 .../join_inner_using_single_col.explain|   2 +-
 .../join_using_multiple_col_array.explain  |   2 +-
 .../join_using_multiple_col_seq.explain|   2 +-
 .../explain-results/join_using_single_col.explain  |   2 +-
 .../query-tests/queries/function_udf.json  |  96 -
 .../query-tests/queries/function_udf.proto.bin | Bin 11257 -> 0 bytes
 .../query-tests/queries/function_udf_2.12.json |  96 -
 .../queries/function_udf_2.12.proto.bin| Bin 11257 -> 0 bytes
 .../query-tests/queries/function_udf_2.13.json |  96 -
 .../queries/function_udf_2.13.proto.bin| Bin 12092 -> 0 bytes
 .../sql/connect/ProtoToParsedPlanTestSuite.scala   |   5 --
 .../spark/sql/catalyst/analysis/unresolved.scala   |   4 +-
 .../spark/sql/catalyst/plans/joinTypes.scala   |   1 +
 21 files changed, 11 insertions(+), 330 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
index f8bae27cc0e..10aa4b658da 100644
--- 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
+++ 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
@@ -20,7 +20,6 @@ import java.nio.file.{Files, Path}
 
 import scala.collection.mutable
 import scala.util.{Failure, Success, Try}
-import scala.util.Properties.versionNumberString
 
 import com.google.protobuf.util.JsonFormat
 import io.grpc.inprocess.InProcessChannelBuilder
@@ -58,8 +57,6 @@ class PlanGenerationTestSuite extends ConnectFunSuite with 
BeforeAndAfterAll wit
   // Borrowed from SparkFunSuite
   private val regenerateGoldenFiles: Boolean = 
System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1"
 
-  private val scala = versionNumberString.substring(0, 
versionNumberString.indexOf(".", 2))
-
   // Borrowed from SparkFunSuite
   private def getWorkspaceFilePath(first: String, more: String*): Path = {
 if (!(sys.props.contains("spark.test.home") || 
sys.env.contains("SPARK_HOME"))) {
@@ -787,24 +784,6 @@ class PlanGenerationTestSuite extends ConnectFunSuite with 
BeforeAndAfterAll wit
 select(fn.max(Column("id")))
   }
 
-  test("function udf " + scala) {
-// This test might be a bit tricky if different JVM
-// versions are used to generate the golden files.
-val functions = Seq(
-  fn.udf(TestUDFs.udf0)
-.asNonNullable()
-.asNondeterministic(),
-  fn.udf(TestUDFs.udf1).withName("foo"),
-  fn.udf(TestUDFs.udf2).withName("f3"),
-  fn.udf(TestUDFs.udf3).withName("bar"),
-  fn.udf(TestUDFs.udf4).withName("f_four"))
-val id = fn.col("id")
-val columns = functions.zipWithIndex.map { case (udf, i) =>
-  udf(Seq.fill(i)(id): _*)
-}
-select(columns: _*)
-  }
-
   test("function 

[spark] branch master updated: [SPARK-42464][CONNECT] Fix ProtoToPlanTestSuite for Scala 2.13

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

hvanhovell 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 229807efd06 [SPARK-42464][CONNECT] Fix ProtoToPlanTestSuite for Scala 
2.13
229807efd06 is described below

commit 229807efd06562fc07fce9a257062d8a66068761
Author: Herman van Hovell 
AuthorDate: Thu Feb 16 14:47:05 2023 -0400

[SPARK-42464][CONNECT] Fix ProtoToPlanTestSuite for Scala 2.13

### What changes were proposed in this pull request?
The `ProtoToPlanTestSuite` were broken for Scala 2.13 . This was caused by 
the following two problems:
- Explain output between 2.12 and 2.13 is not stable because we render 
collection implementations as well. For this I changed the rendering of the 
offending classes to be version agnostic.
- UDF code had deserialization issues. This was always the risk. I have 
removed those tests, we will work on improving UDF coverage in a follow-up.

### Why are the changes needed?
We want to test Scala 2.13 as well.

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

### How was this patch tested?
Test only change.

Closes #40055 from hvanhovell/SPARK-42464.

Authored-by: Herman van Hovell 
Signed-off-by: Herman van Hovell 
---
 .../apache/spark/sql/PlanGenerationTestSuite.scala |  21 -
 .../explain-results/column_as_multi.explain|   2 +-
 .../column_star_with_target.explain|   2 +-
 .../explain-results/function_udf.explain   |   2 -
 .../explain-results/function_udf_2.12.explain  |   2 -
 .../explain-results/function_udf_2.13.explain  |   2 -
 .../join_inner_using_multiple_col_array.explain|   2 +-
 .../join_inner_using_multiple_col_seq.explain  |   2 +-
 .../join_inner_using_single_col.explain|   2 +-
 .../join_using_multiple_col_array.explain  |   2 +-
 .../join_using_multiple_col_seq.explain|   2 +-
 .../explain-results/join_using_single_col.explain  |   2 +-
 .../query-tests/queries/function_udf.json  |  96 -
 .../query-tests/queries/function_udf.proto.bin | Bin 11257 -> 0 bytes
 .../query-tests/queries/function_udf_2.12.json |  96 -
 .../queries/function_udf_2.12.proto.bin| Bin 11257 -> 0 bytes
 .../query-tests/queries/function_udf_2.13.json |  96 -
 .../queries/function_udf_2.13.proto.bin| Bin 12092 -> 0 bytes
 .../sql/connect/ProtoToParsedPlanTestSuite.scala   |   5 --
 .../spark/sql/catalyst/analysis/unresolved.scala   |   4 +-
 .../spark/sql/catalyst/plans/joinTypes.scala   |   1 +
 21 files changed, 11 insertions(+), 330 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
index f8bae27cc0e..10aa4b658da 100644
--- 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
+++ 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala
@@ -20,7 +20,6 @@ import java.nio.file.{Files, Path}
 
 import scala.collection.mutable
 import scala.util.{Failure, Success, Try}
-import scala.util.Properties.versionNumberString
 
 import com.google.protobuf.util.JsonFormat
 import io.grpc.inprocess.InProcessChannelBuilder
@@ -58,8 +57,6 @@ class PlanGenerationTestSuite extends ConnectFunSuite with 
BeforeAndAfterAll wit
   // Borrowed from SparkFunSuite
   private val regenerateGoldenFiles: Boolean = 
System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1"
 
-  private val scala = versionNumberString.substring(0, 
versionNumberString.indexOf(".", 2))
-
   // Borrowed from SparkFunSuite
   private def getWorkspaceFilePath(first: String, more: String*): Path = {
 if (!(sys.props.contains("spark.test.home") || 
sys.env.contains("SPARK_HOME"))) {
@@ -787,24 +784,6 @@ class PlanGenerationTestSuite extends ConnectFunSuite with 
BeforeAndAfterAll wit
 select(fn.max(Column("id")))
   }
 
-  test("function udf " + scala) {
-// This test might be a bit tricky if different JVM
-// versions are used to generate the golden files.
-val functions = Seq(
-  fn.udf(TestUDFs.udf0)
-.asNonNullable()
-.asNondeterministic(),
-  fn.udf(TestUDFs.udf1).withName("foo"),
-  fn.udf(TestUDFs.udf2).withName("f3"),
-  fn.udf(TestUDFs.udf3).withName("bar"),
-  fn.udf(TestUDFs.udf4).withName("f_four"))
-val id = fn.col("id")
-val columns = functions.zipWithIndex.map { case (udf, i) =>
-  udf(Seq.fill(i)(id): _*)
-}
-select(columns: _*)
-  }
-
   test("function lit") {
 select(
   fn.lit(fn.col("id")),
diff --git 

[GitHub] [spark-website] srowen commented on pull request #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


srowen commented on PR #435:
URL: https://github.com/apache/spark-website/pull/435#issuecomment-1433528874

   LOL I have no idea how to meaningfully review it, but at least the number of 
files changed (~3M?!) looks like what happened last time, so OK


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[spark] branch branch-3.4 updated: [SPARK-42287][CONNECT][BUILD] Fix shading so that the JVM client jar can include all 3rd-party dependencies in the runtime

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 130c82c8c5a [SPARK-42287][CONNECT][BUILD] Fix shading so that the JVM 
client jar can include all 3rd-party dependencies in the runtime
130c82c8c5a is described below

commit 130c82c8c5aa56bc1ed94553415225cfd25fe4c0
Author: Zhen Li 
AuthorDate: Thu Feb 16 14:26:01 2023 -0400

[SPARK-42287][CONNECT][BUILD] Fix shading so that the JVM client jar can 
include all 3rd-party dependencies in the runtime

### What changes were proposed in this pull request?
Fix the JVM client dependencies and shading result.
The common jar should not be shaded. The shading should be done in client 
or server.
The common jar shall depends on minimal dependency if possible.
The client jar should provides all compiled dependencies out of the box, 
including netty etc.
The client sbt and mvn shall produce the same shading result.

The current client dependency summary:
```
[INFO] --- maven-dependency-plugin:3.3.0:tree (default-cli)  
spark-connect-client-jvm_2.12 ---
[INFO] org.apache.spark:spark-connect-client-jvm_2.12:jar:3.5.0-SNAPSHOT
[INFO] +- 
org.apache.spark:spark-connect-common_2.12:jar:3.5.0-SNAPSHOT:compile
[INFO] |  +- org.scala-lang:scala-library:jar:2.12.17:compile
[INFO] |  +- io.grpc:grpc-netty:jar:1.47.0:compile
[INFO] |  |  +- io.grpc:grpc-core:jar:1.47.0:compile
[INFO] |  |  |  +- com.google.code.gson:gson:jar:2.9.0:runtime
[INFO] |  |  |  +- com.google.android:annotations:jar:4.1.1.4:runtime
[INFO] |  |  |  \- 
org.codehaus.mojo:animal-sniffer-annotations:jar:1.19:runtime
[INFO] |  |  \- io.perfmark:perfmark-api:jar:0.25.0:runtime
[INFO] |  +- io.grpc:grpc-protobuf:jar:1.47.0:compile
[INFO] |  |  +- io.grpc:grpc-api:jar:1.47.0:compile
[INFO] |  |  |  \- io.grpc:grpc-context:jar:1.47.0:compile
[INFO] |  |  +- 
com.google.api.grpc:proto-google-common-protos:jar:2.0.1:compile
[INFO] |  |  \- io.grpc:grpc-protobuf-lite:jar:1.47.0:compile
[INFO] |  +- io.grpc:grpc-services:jar:1.47.0:compile
[INFO] |  |  \- com.google.protobuf:protobuf-java-util:jar:3.19.2:runtime
[INFO] |  \- io.grpc:grpc-stub:jar:1.47.0:compile
[INFO] +- com.google.protobuf:protobuf-java:jar:3.21.12:compile
[INFO] +- com.google.guava:guava:jar:31.0.1-jre:compile
[INFO] |  +- com.google.guava:failureaccess:jar:1.0.1:compile
[INFO] |  +- 
com.google.guava:listenablefuture:jar:.0-empty-to-avoid-conflict-with-guava:compile
[INFO] |  +- com.google.code.findbugs:jsr305:jar:3.0.0:compile
[INFO] |  +- org.checkerframework:checker-qual:jar:3.12.0:compile
[INFO] |  +- com.google.errorprone:error_prone_annotations:jar:2.7.1:compile
[INFO] |  \- com.google.j2objc:j2objc-annotations:jar:1.3:compile
[INFO] +- io.netty:netty-codec-http2:jar:4.1.87.Final:compile
[INFO] |  +- io.netty:netty-common:jar:4.1.87.Final:compile
[INFO] |  +- io.netty:netty-buffer:jar:4.1.87.Final:compile
[INFO] |  +- io.netty:netty-transport:jar:4.1.87.Final:compile
[INFO] |  |  \- io.netty:netty-resolver:jar:4.1.87.Final:compile
[INFO] |  +- io.netty:netty-codec:jar:4.1.87.Final:compile
[INFO] |  +- io.netty:netty-handler:jar:4.1.87.Final:compile
[INFO] |  \- io.netty:netty-codec-http:jar:4.1.87.Final:compile
[INFO] +- io.netty:netty-handler-proxy:jar:4.1.87.Final:compile
[INFO] |  \- io.netty:netty-codec-socks:jar:4.1.87.Final:compile
[INFO] +- 
io.netty:netty-transport-native-unix-common:jar:4.1.87.Final:compile
[INFO] +- org.spark-project.spark:unused:jar:1.0.0:compile
```

### Why are the changes needed?

Fix the client jar package.

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

No.

### How was this patch tested?

Existing tests.

Closes #39866 from zhenlineo/fix-jars.

Authored-by: Zhen Li 
Signed-off-by: Herman van Hovell 
(cherry picked from commit 49af23aa87d7a566e16c81afbcfb49c0e2064536)
Signed-off-by: Herman van Hovell 
---
 connector/connect/client/jvm/pom.xml | 96 +++-
 connector/connect/common/pom.xml | 48 --
 project/SparkBuild.scala | 14 --
 3 files changed, 72 insertions(+), 86 deletions(-)

diff --git a/connector/connect/client/jvm/pom.xml 
b/connector/connect/client/jvm/pom.xml
index 593819f7108..a3acf046dbc 100644
--- a/connector/connect/client/jvm/pom.xml
+++ b/connector/connect/client/jvm/pom.xml
@@ -33,6 +33,7 @@
   
 connect-client-jvm
 31.0.1-jre
+1.0.1
 1.1.0
   
 
@@ -53,6 +54,13 @@
   org.apache.spark
   spark-catalyst_${scala.binary.version}
   ${project.version}
+  provided
+  
+
+  com.google.guava
+  

[spark] branch master updated (8d863e306cb -> 49af23aa87d)

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

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


from 8d863e306cb [SPARK-42457][CONNECT] Adding SparkSession#read
 add 49af23aa87d [SPARK-42287][CONNECT][BUILD] Fix shading so that the JVM 
client jar can include all 3rd-party dependencies in the runtime

No new revisions were added by this update.

Summary of changes:
 connector/connect/client/jvm/pom.xml | 96 +++-
 connector/connect/common/pom.xml | 48 --
 project/SparkBuild.scala | 14 --
 3 files changed, 72 insertions(+), 86 deletions(-)


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



[spark] branch branch-3.4 updated: [SPARK-42457][CONNECT] Adding SparkSession#read

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

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


The following commit(s) were added to refs/heads/branch-3.4 by this push:
 new 53f11b1087a [SPARK-42457][CONNECT] Adding SparkSession#read
53f11b1087a is described below

commit 53f11b1087a192af5efadc36af7413b0c21c87e1
Author: Zhen Li 
AuthorDate: Thu Feb 16 14:22:47 2023 -0400

[SPARK-42457][CONNECT] Adding SparkSession#read

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

Add SparkSession Read API to read data into Spark via Scala Client:
```
DataFrameReader.format(…).option(“key”, “value”).schema(…).load()
```

The following methods are skipped by the Scala Client on purpose:
```
[info]   deprecated method 
json(org.apache.spark.api.java.JavaRDD)org.apache.spark.sql.Dataset in class 
org.apache.spark.sql.DataFrameReader does not have a correspondent in client 
version
[info]   deprecated method 
json(org.apache.spark.rdd.RDD)org.apache.spark.sql.Dataset in class 
org.apache.spark.sql.DataFrameReader does not have a correspondent in client 
version
[info]   method 
json(org.apache.spark.sql.Dataset)org.apache.spark.sql.Dataset in class 
org.apache.spark.sql.DataFrameReader does not have a correspondent in client 
version
[info]   method 
csv(org.apache.spark.sql.Dataset)org.apache.spark.sql.Dataset in class 
org.apache.spark.sql.DataFrameReader does not have a correspondent in client 
version
```

### Why are the changes needed?
To read data from csv etc. format.

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

### How was this patch tested?
E2E, Golden tests.

Closes #40025 from zhenlineo/session-read.

Authored-by: Zhen Li 
Signed-off-by: Herman van Hovell 
(cherry picked from commit 8d863e306cb105b715c8b9206a2bdd944dafa90b)
Signed-off-by: Herman van Hovell 
---
 .../org/apache/spark/sql/DataFrameReader.scala | 409 +
 .../scala/org/apache/spark/sql/SparkSession.scala  |  29 ++
 .../org/apache/spark/sql/ClientE2ETestSuite.scala  |  66 +++-
 .../apache/spark/sql/PlanGenerationTestSuite.scala |  54 ++-
 .../sql/connect/client/CompatibilitySuite.scala|   7 +-
 .../connect/client/util/IntegrationTestUtils.scala |   2 +-
 .../query-tests/explain-results/read.explain   |   1 +
 .../query-tests/explain-results/read_csv.explain   |   1 +
 .../query-tests/explain-results/read_json.explain  |   1 +
 .../query-tests/explain-results/read_orc.explain   |   1 +
 .../explain-results/read_parquet.explain   |   1 +
 .../query-tests/explain-results/read_path.explain  |   1 +
 .../query-tests/explain-results/read_table.explain |   1 +
 .../query-tests/explain-results/read_text.explain  |   1 +
 .../query-tests/explain-results/table.explain  |   1 +
 .../test/resources/query-tests/queries/read.json   |  13 +
 .../resources/query-tests/queries/read.proto.bin   |   4 +
 .../resources/query-tests/queries/read_csv.json|   8 +
 .../query-tests/queries/read_csv.proto.bin |   2 +
 .../resources/query-tests/queries/read_json.json   |   8 +
 .../query-tests/queries/read_json.proto.bin|   2 +
 .../resources/query-tests/queries/read_orc.json|   8 +
 .../query-tests/queries/read_orc.proto.bin |   2 +
 .../query-tests/queries/read_parquet.json  |   8 +
 .../query-tests/queries/read_parquet.proto.bin |   2 +
 .../resources/query-tests/queries/read_path.json   |  11 +
 .../query-tests/queries/read_path.proto.bin|   3 +
 .../resources/query-tests/queries/read_table.json  |   7 +
 .../query-tests/queries/read_table.proto.bin   |   3 +
 .../resources/query-tests/queries/read_text.json   |   8 +
 .../query-tests/queries/read_text.proto.bin|   2 +
 .../test/resources/query-tests/queries/table.json  |   7 +
 .../resources/query-tests/queries/table.proto.bin  |   3 +
 .../resources/query-tests/test-data/people.csv |   3 +
 .../resources/query-tests/test-data/people.json|   3 +
 .../resources/query-tests/test-data/people.txt |   3 +
 .../test/resources/query-tests/test-data/users.orc | Bin 0 -> 547 bytes
 .../resources/query-tests/test-data/users.parquet  | Bin 0 -> 615 bytes
 38 files changed, 681 insertions(+), 5 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
new file mode 100644
index 000..5a486efee31
--- /dev/null
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -0,0 +1,409 @@
+/*
+ * 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 

[spark] branch master updated: [SPARK-42457][CONNECT] Adding SparkSession#read

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

hvanhovell 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 8d863e306cb [SPARK-42457][CONNECT] Adding SparkSession#read
8d863e306cb is described below

commit 8d863e306cb105b715c8b9206a2bdd944dafa90b
Author: Zhen Li 
AuthorDate: Thu Feb 16 14:22:47 2023 -0400

[SPARK-42457][CONNECT] Adding SparkSession#read

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

Add SparkSession Read API to read data into Spark via Scala Client:
```
DataFrameReader.format(…).option(“key”, “value”).schema(…).load()
```

The following methods are skipped by the Scala Client on purpose:
```
[info]   deprecated method 
json(org.apache.spark.api.java.JavaRDD)org.apache.spark.sql.Dataset in class 
org.apache.spark.sql.DataFrameReader does not have a correspondent in client 
version
[info]   deprecated method 
json(org.apache.spark.rdd.RDD)org.apache.spark.sql.Dataset in class 
org.apache.spark.sql.DataFrameReader does not have a correspondent in client 
version
[info]   method 
json(org.apache.spark.sql.Dataset)org.apache.spark.sql.Dataset in class 
org.apache.spark.sql.DataFrameReader does not have a correspondent in client 
version
[info]   method 
csv(org.apache.spark.sql.Dataset)org.apache.spark.sql.Dataset in class 
org.apache.spark.sql.DataFrameReader does not have a correspondent in client 
version
```

### Why are the changes needed?
To read data from csv etc. format.

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

### How was this patch tested?
E2E, Golden tests.

Closes #40025 from zhenlineo/session-read.

Authored-by: Zhen Li 
Signed-off-by: Herman van Hovell 
---
 .../org/apache/spark/sql/DataFrameReader.scala | 409 +
 .../scala/org/apache/spark/sql/SparkSession.scala  |  29 ++
 .../org/apache/spark/sql/ClientE2ETestSuite.scala  |  66 +++-
 .../apache/spark/sql/PlanGenerationTestSuite.scala |  54 ++-
 .../sql/connect/client/CompatibilitySuite.scala|   7 +-
 .../connect/client/util/IntegrationTestUtils.scala |   2 +-
 .../query-tests/explain-results/read.explain   |   1 +
 .../query-tests/explain-results/read_csv.explain   |   1 +
 .../query-tests/explain-results/read_json.explain  |   1 +
 .../query-tests/explain-results/read_orc.explain   |   1 +
 .../explain-results/read_parquet.explain   |   1 +
 .../query-tests/explain-results/read_path.explain  |   1 +
 .../query-tests/explain-results/read_table.explain |   1 +
 .../query-tests/explain-results/read_text.explain  |   1 +
 .../query-tests/explain-results/table.explain  |   1 +
 .../test/resources/query-tests/queries/read.json   |  13 +
 .../resources/query-tests/queries/read.proto.bin   |   4 +
 .../resources/query-tests/queries/read_csv.json|   8 +
 .../query-tests/queries/read_csv.proto.bin |   2 +
 .../resources/query-tests/queries/read_json.json   |   8 +
 .../query-tests/queries/read_json.proto.bin|   2 +
 .../resources/query-tests/queries/read_orc.json|   8 +
 .../query-tests/queries/read_orc.proto.bin |   2 +
 .../query-tests/queries/read_parquet.json  |   8 +
 .../query-tests/queries/read_parquet.proto.bin |   2 +
 .../resources/query-tests/queries/read_path.json   |  11 +
 .../query-tests/queries/read_path.proto.bin|   3 +
 .../resources/query-tests/queries/read_table.json  |   7 +
 .../query-tests/queries/read_table.proto.bin   |   3 +
 .../resources/query-tests/queries/read_text.json   |   8 +
 .../query-tests/queries/read_text.proto.bin|   2 +
 .../test/resources/query-tests/queries/table.json  |   7 +
 .../resources/query-tests/queries/table.proto.bin  |   3 +
 .../resources/query-tests/test-data/people.csv |   3 +
 .../resources/query-tests/test-data/people.json|   3 +
 .../resources/query-tests/test-data/people.txt |   3 +
 .../test/resources/query-tests/test-data/users.orc | Bin 0 -> 547 bytes
 .../resources/query-tests/test-data/users.parquet  | Bin 0 -> 615 bytes
 38 files changed, 681 insertions(+), 5 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
new file mode 100644
index 000..5a486efee31
--- /dev/null
+++ 
b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
@@ -0,0 +1,409 @@
+/*
+ * 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 

[GitHub] [spark-website] viirya opened a new pull request, #435: Add docs for Apache Spark 3.3.2

2023-02-16 Thread via GitHub


viirya opened a new pull request, #435:
URL: https://github.com/apache/spark-website/pull/435

   Adds docs from https://dist.apache.org/repos/dist/dev/spark/v3.3.2-rc1-docs/


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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



[spark] branch master updated: [SPARK-42463][SPARK-27180][YARN][TESTS] Clean up the third-party Java files copy introduced by

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

srowen 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 64e5928e863 [SPARK-42463][SPARK-27180][YARN][TESTS] Clean up the 
third-party Java files copy introduced by
64e5928e863 is described below

commit 64e5928e86377a216e48bb7be12150ad130322f3
Author: yangjie01 
AuthorDate: Thu Feb 16 09:26:33 2023 -0600

[SPARK-42463][SPARK-27180][YARN][TESTS] Clean up the third-party Java files 
copy introduced by

### What changes were proposed in this pull request?
SPARK-27180 introduced some third-party Java source code  to solve Yarn 
module test failure,  but maven and sbt can also test pass without them, so 
this pr remove these files.

### Why are the changes needed?
Clean up the third-party Java source code copy in Spark.

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

### How was this patch tested?
- Pass GitHub Actions
- manual check:

**Maven**

```
build/mvn clean
build/mvn clean install -DskipTestes -pl resource-managers/yarn -am -Pyarn
build/mvn -Dtest=none 
-DwildcardSuites=org.apache.spark.deploy.yarn.YarnClusterSuite -pl 
resource-managers/yarn test -Pyarn
build/mvn test -pl resource-managers/yarn -Pyarn 
-Dtest.exclude.tags=org.apache.spark.tags.ExtendedLevelDBTest
```
Both `YarnClusterSuite` and full module test passed.

**SBT**

```
build/sbt clean yarn/test -Pyarn 
-Dtest.exclude.tags=org.apache.spark.tags.ExtendedLevelDBTest
```
All tests passed.

Closes #40052 from LuciferYang/SPARK-42463.

Authored-by: yangjie01 
Signed-off-by: Sean Owen 
---
 dev/.rat-excludes  |   2 -
 .../org/apache/hadoop/net/ServerSocketUtil.java| 132 --
 .../org/eclipse/jetty/server/SessionManager.java   | 290 -
 .../jetty/server/session/SessionHandler.java   |  90 ---
 4 files changed, 514 deletions(-)

diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index 458fe7594b2..9ac820fc216 100644
--- a/dev/.rat-excludes
+++ b/dev/.rat-excludes
@@ -118,8 +118,6 @@ kafka-source-initial-offset-version-2.1.0.bin
 kafka-source-initial-offset-future-version.bin
 announce.tmpl
 vote.tmpl
-SessionManager.java
-SessionHandler.java
 GangliaReporter.java
 application_1578436911597_0052
 config.properties
diff --git 
a/resource-managers/yarn/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
 
b/resource-managers/yarn/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
deleted file mode 100644
index 89e012ecd42..000
--- 
a/resource-managers/yarn/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.net;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.ServerSocket;
-import java.util.Random;
-
-/**
- * Copied from
- * 
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java
- * for Hadoop-3.x testing
- */
-public class ServerSocketUtil {
-
-  private static final Logger LOG = 
LoggerFactory.getLogger(ServerSocketUtil.class);
-  private static Random rand = new Random();
-
-  /**
-   * Port scan & allocate is how most other apps find ports
-   *
-   * @param port given port
-   * @param retries number of retries
-   * @return
-   * @throws IOException
-   */
-  public static int getPort(int port, int retries) throws IOException {
-int tryPort = port;
-int tries = 0;
-while (true) {
-  if (tries > 0 || tryPort == 0) {
-tryPort = port + rand.nextInt(65535 - port);
-  }
-  if (tryPort == 0) {
-continue;
-  }
-  try (ServerSocket s = new ServerSocket(tryPort)) {
-LOG.info("Using port " + tryPort);
-return tryPort;
-  } catch (IOException e) {
-tries++;
-if (tries >= retries) {
-  LOG.info("Port is already in use; giving up");
-  throw e;
-} else {

[spark] branch master updated: [SPARK-42424][YARN] Remove unused declarations from Yarn module

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

srowen 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 f5deb7c9af6 [SPARK-42424][YARN] Remove unused declarations  from Yarn 
module
f5deb7c9af6 is described below

commit f5deb7c9af618494adecfeb103f94219e27fcf70
Author: yangjie01 
AuthorDate: Thu Feb 16 09:25:47 2023 -0600

[SPARK-42424][YARN] Remove unused declarations  from Yarn module

### What changes were proposed in this pull request?
This pr cleans up unused declarations in the Yarn module:

- `YarnSparkHadoopUtil#setEnvFromInputString`: this method Introduced by 
SPARK-3477 and becomes a unused and no test coverage method after SPARK-17979

- `YarnSparkHadoopUtil#environmentVariableRegex`: this `val` is only used 
by `YarnSparkHadoopUtil#setEnvFromInputString`

- `ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS`: this `val` 
Introduced by SPARK-1946 and replaced by 
`YarnSparkHadoopUtil.DEFAULT_NUMBER_EXECUTORS` in SPARK-4138, and It was 
eventually replaced by `config#EXECUTOR_INSTANCES`

- `ApplicationMaster.EXIT_SECURITY`: this `val` introduced by SPARK-3627 
and it is used to represent the exit code of errors related to 
`System.setSecurityManager`, SPARK-4584 deleted the use of `SecurityManager` 
and this val is useless.

### Why are the changes needed?
Code clean up.

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

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

Closes #39997 from LuciferYang/SPARK-42424.

Authored-by: yangjie01 
Signed-off-by: Sean Owen 
---
 .../spark/deploy/yarn/ApplicationMaster.scala  |  1 -
 .../deploy/yarn/ApplicationMasterArguments.scala   |  4 --
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala| 50 --
 3 files changed, 55 deletions(-)

diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 9815fa6df8a..252c84a1cd4 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -891,7 +891,6 @@ object ApplicationMaster extends Logging {
   private val EXIT_MAX_EXECUTOR_FAILURES = 11
   private val EXIT_REPORTER_FAILURE = 12
   private val EXIT_SC_NOT_INITED = 13
-  private val EXIT_SECURITY = 14
   private val EXIT_EXCEPTION_USER_CLASS = 15
   private val EXIT_EARLY = 16
   private val EXIT_DISCONNECTED = 17
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
index d2275980814..821a31502af 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -102,7 +102,3 @@ class ApplicationMasterArguments(val args: Array[String]) {
 System.exit(exitCode)
   }
 }
-
-object ApplicationMasterArguments {
-  val DEFAULT_NUMBER_EXECUTORS = 2
-}
diff --git 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
index 1869c739e48..4d050b91a85 100644
--- 
a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ 
b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -17,8 +17,6 @@
 
 package org.apache.spark.deploy.yarn
 
-import java.util.regex.{Matcher, Pattern}
-
 import scala.collection.immutable.{Map => IMap}
 import scala.collection.mutable.{HashMap, ListBuffer}
 import scala.util.matching.Regex
@@ -59,41 +57,6 @@ object YarnSparkHadoopUtil {
 env.put(key, newValue)
   }
 
-  /**
-   * Set zero or more environment variables specified by the given input 
string.
-   * The input string is expected to take the form 
"KEY1=VAL1,KEY2=VAL2,KEY3=VAL3".
-   */
-  def setEnvFromInputString(env: HashMap[String, String], inputString: 
String): Unit = {
-if (inputString != null && inputString.length() > 0) {
-  val childEnvs = inputString.split(",")
-  val p = Pattern.compile(environmentVariableRegex)
-  for (cEnv <- childEnvs) {
-val parts = cEnv.split("=") // split on '='
-val m = p.matcher(parts(1))
-val sb = new StringBuffer
-while (m.find()) {
-  val variable = m.group(1)
-  var replace = ""
-  if (env.contains(variable)) {
-replace = env(variable)
-  } else {
-