[spark] branch master updated: [SPARK-43956][SQL] Fix the bug doesn't display column's sql for Percentile[Cont|Disc]
This is an automated email from the ASF dual-hosted git repository. yumwang 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 898ad77900d [SPARK-43956][SQL] Fix the bug doesn't display column's sql for Percentile[Cont|Disc] 898ad77900d is described below commit 898ad77900d887ac64800a616bd382def816eea6 Author: Jiaan Geng AuthorDate: Sat Jun 3 13:19:09 2023 +0800 [SPARK-43956][SQL] Fix the bug doesn't display column's sql for Percentile[Cont|Disc] ### What changes were proposed in this pull request? Last year, I committed `Percentile[Cont|Disc]` functions for Spark SQL. Recently, I found the sql method of `Percentile[Cont|Disc]` doesn't display column's sql suitably. This PR will fix the bug. ### Why are the changes needed? Fix the bug doesn't display column's sql for `Percentile[Cont|Disc]`. ### Does this PR introduce _any_ user-facing change? 'Yes'. Users could see the correct sql information. ### How was this patch tested? Test cases updated. Closes #41436 from beliefer/SPARK-37676_SPARK-37691_followup. Authored-by: Jiaan Geng Signed-off-by: Yuming Wang --- .../expressions/aggregate/percentiles.scala| 4 +-- .../sql-tests/analyzer-results/percentiles.sql.out | 36 +++--- .../postgreSQL/aggregates_part4.sql.out| 8 ++--- .../udf/postgreSQL/udf-aggregates_part4.sql.out| 8 ++--- .../sql-tests/results/percentiles.sql.out | 24 +++ .../results/postgreSQL/aggregates_part4.sql.out| 8 ++--- .../udf/postgreSQL/udf-aggregates_part4.sql.out| 8 ++--- 7 files changed, 48 insertions(+), 48 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala index 6d54c52b918..b26502a1763 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/percentiles.scala @@ -364,7 +364,7 @@ case class PercentileCont(left: Expression, right: Expression, reverse: Boolean override def sql(isDistinct: Boolean): String = { val distinct = if (isDistinct) "DISTINCT " else "" val direction = if (reverse) " DESC" else "" -s"$prettyName($distinct${right.sql}) WITHIN GROUP (ORDER BY v$direction)" +s"$prettyName($distinct${right.sql}) WITHIN GROUP (ORDER BY ${left.sql}$direction)" } override protected def withNewChildrenInternal( newLeft: Expression, newRight: Expression): PercentileCont = @@ -404,7 +404,7 @@ case class PercentileDisc( override def sql(isDistinct: Boolean): String = { val distinct = if (isDistinct) "DISTINCT " else "" val direction = if (reverse) " DESC" else "" -s"$prettyName($distinct${right.sql}) WITHIN GROUP (ORDER BY v$direction)" +s"$prettyName($distinct${right.sql}) WITHIN GROUP (ORDER BY ${left.sql}$direction)" } override protected def withNewChildrenInternal( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out index 9c945687830..7a72df5a3e3 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/percentiles.sql.out @@ -182,9 +182,9 @@ FROM basic_pays ORDER BY salary -- !query analysis Sort [salary#x ASC NULLS FIRST], true -+- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0 [...] - +- Project [employee_name#x, department#x, salary#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_disc(0.25) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, percentile_dis [...] - +- Window [percentile_cont(salary#x, cast(0.25 as double), false) windowspecdefinition(department#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS percentile_cont(0.25) WITHIN GROUP (ORDER BY v) OVE
[spark] branch master updated: [SPARK-43380][SQL] Fix Avro data type conversion issues to avoid producing incorrect results
This is an automated email from the ASF dual-hosted git repository. gengliang 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 14b00cfc6c2 [SPARK-43380][SQL] Fix Avro data type conversion issues to avoid producing incorrect results 14b00cfc6c2 is described below commit 14b00cfc6c2e477067fc9e7937e34b2aa53df1eb Author: zeruibao AuthorDate: Fri Jun 2 15:10:06 2023 -0700 [SPARK-43380][SQL] Fix Avro data type conversion issues to avoid producing incorrect results ### What changes were proposed in this pull request? We introduce the SQLConf `spark.sql.legacy.avro.allowReadingWithIncompatibleSchema` to prevent reading interval types as date or timestamp types to avoid getting corrupt dates as well as reading decimal types with incorrect precision. ### Why are the changes needed? We found the following issues with open source Avro: - Interval types can be read as date or timestamp types that would lead to wildly different results For example, `Duration.ofDays(1).plusSeconds(1)` will be read as `1972-09-27`, which is weird. - Decimal types can be read with lower precision, that leads to data being read as `null` instead of suggesting that a wider decimal format should be provided ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? New unit test Closes #41052 from zeruibao/SPARK-43380-fix-avro-data-type-conversion. Lead-authored-by: zeruibao Co-authored-by: zeruibao <125398515+zerui...@users.noreply.github.com> Signed-off-by: Gengliang Wang --- .../apache/spark/sql/avro/AvroDeserializer.scala | 423 - .../org/apache/spark/sql/avro/AvroSuite.scala | 132 +++ core/src/main/resources/error/error-classes.json | 10 + docs/sql-migration-guide.md| 1 + .../spark/sql/errors/QueryCompilationErrors.scala | 30 ++ .../org/apache/spark/sql/internal/SQLConf.scala| 12 + 6 files changed, 435 insertions(+), 173 deletions(-) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala index aac979cddb2..78b1f01e2ef 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala @@ -35,7 +35,9 @@ import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArr import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_DAY import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.DataSourceUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -117,178 +119,260 @@ private[sql] class AvroDeserializer( val incompatibleMsg = errorPrefix + s"schema is incompatible (avroType = $avroType, sqlType = ${catalystType.sql})" -(avroType.getType, catalystType) match { - case (NULL, NullType) => (updater, ordinal, _) => -updater.setNullAt(ordinal) +val confKey = SQLConf.LEGACY_AVRO_ALLOW_INCOMPATIBLE_SCHEMA +val preventReadingIncorrectType = !SQLConf.get.getConf(confKey) +val logicalDataType = SchemaConverters.toSqlType(avroType).dataType +avroType.getType match { + case NULL => +(logicalDataType, catalystType) match { + case (_, NullType) => (updater, ordinal, _) => +updater.setNullAt(ordinal) + case _ => throw new IncompatibleSchemaException(incompatibleMsg) +} // TODO: we can avoid boxing if future version of avro provide primitive accessors. - case (BOOLEAN, BooleanType) => (updater, ordinal, value) => -updater.setBoolean(ordinal, value.asInstanceOf[Boolean]) - - case (INT, IntegerType) => (updater, ordinal, value) => -updater.setInt(ordinal, value.asInstanceOf[Int]) - - case (INT, DateType) => (updater, ordinal, value) => -updater.setInt(ordinal, dateRebaseFunc(value.asInstanceOf[Int])) - - case (LONG, LongType) => (updater, ordinal, value) => -updater.setLong(ordinal, value.asInstanceOf[Long]) - - case (LONG, TimestampType) => avroType.getLogicalType match { -// For backward compatibility, if the Avro type is Long and it is not logical type -// (the `null` case), the value is processed as timestamp type with millisecond precision. -case null | _: TimestampMillis => (updater, ordina
[spark] branch master updated: [SPARK-36612][SQL] Support left outer join build left or right outer join build right in shuffled hash join
This is an automated email from the ASF dual-hosted git repository. huaxingao 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 0effbec16ed [SPARK-36612][SQL] Support left outer join build left or right outer join build right in shuffled hash join 0effbec16ed is described below commit 0effbec16edc27c644d4089bdf266cd4ecbed235 Author: Szehon Ho AuthorDate: Fri Jun 2 10:48:32 2023 -0700 [SPARK-36612][SQL] Support left outer join build left or right outer join build right in shuffled hash join ### What changes were proposed in this pull request? Add support for shuffle-hash join for following scenarios: * left outer join with left-side build * right outer join with right-side build The algorithm is similar to SPARK-32399, which supports shuffle-hash join for full outer join. The same methods fullOuterJoinWithUniqueKey and fullOuterJoinWithNonUniqueKey are improved to support the new case. These methods are called after the HashedRelation is already constructed of the build side, and do these two iterations: 1. Iterate Stream side. a. If find match on build side, mark. b. If no match on build side, join with null build-side row and add to result 2. Iterate build side. a. If find marked for match, add joined row to result b. If no match marked, join with null stream-side row The left outer join with left-side build, and right outer join with right-side build, need only a subset of these logics, namely replacing 1b above with a no-op. Codegen is left for a follow-up PR. ### Why are the changes needed? For joins of these types, shuffle-hash join can be more performant than sort-merge join, especially if the big table is large, as it skips an expensive sort of the big table. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Unit test in JoinSuite.scala Closes #41398 from szehon-ho/same_side_outer_build_join_master. Authored-by: Szehon Ho Signed-off-by: huaxingao --- .../spark/sql/catalyst/optimizer/joins.scala | 4 +- .../spark/sql/execution/joins/HashedRelation.scala | 1 - .../sql/execution/joins/ShuffledHashJoinExec.scala | 74 ++--- .../scala/org/apache/spark/sql/JoinHintSuite.scala | 30 ++--- .../scala/org/apache/spark/sql/JoinSuite.scala | 77 ++ 5 files changed, 151 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 972be43a946..48b4007a897 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -374,14 +374,14 @@ trait JoinSelectionHelper { def canBuildShuffledHashJoinLeft(joinType: JoinType): Boolean = { joinType match { - case _: InnerLike | RightOuter | FullOuter => true + case _: InnerLike | LeftOuter | FullOuter | RightOuter => true case _ => false } } def canBuildShuffledHashJoinRight(joinType: JoinType): Boolean = { joinType match { - case _: InnerLike | LeftOuter | FullOuter | + case _: InnerLike | LeftOuter | FullOuter | RightOuter | LeftSemi | LeftAnti | _: ExistenceJoin => true case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 4d3e63282fa..16345bb35db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -127,7 +127,6 @@ private[execution] object HashedRelation { * Create a HashedRelation from an Iterator of InternalRow. * * @param allowsNullKeyAllow NULL keys in HashedRelation. - * This is used for full outer join in `ShuffledHashJoinExec` only. * @param ignoresDuplicatedKey Ignore rows with duplicated keys in HashedRelation. * This is only used for semi and anti join without join condition in * `ShuffledHashJoinExec` only. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index cfe35d04778..8953bf19f35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@
[spark] branch master updated: [SPARK-43904][BUILD] Upgrade jackson to 2.15.2
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 41d34e562ff [SPARK-43904][BUILD] Upgrade jackson to 2.15.2 41d34e562ff is described below commit 41d34e562ffeb821718e4e1538095e86df7b4f0f Author: panbingkun AuthorDate: Fri Jun 2 09:46:46 2023 -0700 [SPARK-43904][BUILD] Upgrade jackson to 2.15.2 ### What changes were proposed in this pull request? Upgrade FasterXML jackson from 2.15.1 to 2.15.2 ### Why are the changes needed? New version that fix some bugs, release notes as follows: - https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.15.2 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. Closes #41414 from panbingkun/SPARK-43904. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 14 +++--- pom.xml | 4 ++-- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 2b11598c737..d843fa51f97 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -98,13 +98,13 @@ httpcore/4.4.16//httpcore-4.4.16.jar ini4j/0.5.4//ini4j-0.5.4.jar istack-commons-runtime/3.0.8//istack-commons-runtime-3.0.8.jar ivy/2.5.1//ivy-2.5.1.jar -jackson-annotations/2.15.1//jackson-annotations-2.15.1.jar -jackson-core/2.15.1//jackson-core-2.15.1.jar -jackson-databind/2.15.1//jackson-databind-2.15.1.jar -jackson-dataformat-cbor/2.15.1//jackson-dataformat-cbor-2.15.1.jar -jackson-dataformat-yaml/2.15.1//jackson-dataformat-yaml-2.15.1.jar -jackson-datatype-jsr310/2.15.1//jackson-datatype-jsr310-2.15.1.jar -jackson-module-scala_2.12/2.15.1//jackson-module-scala_2.12-2.15.1.jar +jackson-annotations/2.15.2//jackson-annotations-2.15.2.jar +jackson-core/2.15.2//jackson-core-2.15.2.jar +jackson-databind/2.15.2//jackson-databind-2.15.2.jar +jackson-dataformat-cbor/2.15.2//jackson-dataformat-cbor-2.15.2.jar +jackson-dataformat-yaml/2.15.2//jackson-dataformat-yaml-2.15.2.jar +jackson-datatype-jsr310/2.15.2//jackson-datatype-jsr310-2.15.2.jar +jackson-module-scala_2.12/2.15.2//jackson-module-scala_2.12-2.15.2.jar jakarta.annotation-api/1.3.5//jakarta.annotation-api-1.3.5.jar jakarta.inject/2.6.1//jakarta.inject-2.6.1.jar jakarta.servlet-api/4.0.3//jakarta.servlet-api-4.0.3.jar diff --git a/pom.xml b/pom.xml index dc3b65471e4..175df1722e6 100644 --- a/pom.xml +++ b/pom.xml @@ -179,8 +179,8 @@ true true 1.9.13 -2.15.1 - 2.15.1 +2.15.2 + 2.15.2 1.1.10.0 3.0.3 1.15 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43950][BUILD] Upgrade `kubernetes-client` to 6.7.0
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 592857945c4 [SPARK-43950][BUILD] Upgrade `kubernetes-client` to 6.7.0 592857945c4 is described below commit 592857945c45a5db78595bad414f94ef3cd3ac77 Author: Bjørn Jørgensen AuthorDate: Fri Jun 2 09:46:03 2023 -0700 [SPARK-43950][BUILD] Upgrade `kubernetes-client` to 6.7.0 ### What changes were proposed in this pull request? Upgrade kubernetes-client from 6.6.2 to 6.7.0 ### Why are the changes needed? New version that fix some bugs https://github.com/fabric8io/kubernetes-client/releases/tag/v6.7.0 Please note that there are some breaking changes in this version. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA Closes #41434 from bjornjorgensen/k8s6.7.0. Authored-by: Bjørn Jørgensen Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 50 +-- pom.xml | 2 +- 2 files changed, 26 insertions(+), 26 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 830c391046c..2b11598c737 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -141,31 +141,31 @@ jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar jul-to-slf4j/2.0.7//jul-to-slf4j-2.0.7.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client-api/6.6.2//kubernetes-client-api-6.6.2.jar -kubernetes-client/6.6.2//kubernetes-client-6.6.2.jar -kubernetes-httpclient-okhttp/6.6.2//kubernetes-httpclient-okhttp-6.6.2.jar -kubernetes-model-admissionregistration/6.6.2//kubernetes-model-admissionregistration-6.6.2.jar -kubernetes-model-apiextensions/6.6.2//kubernetes-model-apiextensions-6.6.2.jar -kubernetes-model-apps/6.6.2//kubernetes-model-apps-6.6.2.jar -kubernetes-model-autoscaling/6.6.2//kubernetes-model-autoscaling-6.6.2.jar -kubernetes-model-batch/6.6.2//kubernetes-model-batch-6.6.2.jar -kubernetes-model-certificates/6.6.2//kubernetes-model-certificates-6.6.2.jar -kubernetes-model-common/6.6.2//kubernetes-model-common-6.6.2.jar -kubernetes-model-coordination/6.6.2//kubernetes-model-coordination-6.6.2.jar -kubernetes-model-core/6.6.2//kubernetes-model-core-6.6.2.jar -kubernetes-model-discovery/6.6.2//kubernetes-model-discovery-6.6.2.jar -kubernetes-model-events/6.6.2//kubernetes-model-events-6.6.2.jar -kubernetes-model-extensions/6.6.2//kubernetes-model-extensions-6.6.2.jar -kubernetes-model-flowcontrol/6.6.2//kubernetes-model-flowcontrol-6.6.2.jar -kubernetes-model-gatewayapi/6.6.2//kubernetes-model-gatewayapi-6.6.2.jar -kubernetes-model-metrics/6.6.2//kubernetes-model-metrics-6.6.2.jar -kubernetes-model-networking/6.6.2//kubernetes-model-networking-6.6.2.jar -kubernetes-model-node/6.6.2//kubernetes-model-node-6.6.2.jar -kubernetes-model-policy/6.6.2//kubernetes-model-policy-6.6.2.jar -kubernetes-model-rbac/6.6.2//kubernetes-model-rbac-6.6.2.jar -kubernetes-model-resource/6.6.2//kubernetes-model-resource-6.6.2.jar -kubernetes-model-scheduling/6.6.2//kubernetes-model-scheduling-6.6.2.jar -kubernetes-model-storageclass/6.6.2//kubernetes-model-storageclass-6.6.2.jar +kubernetes-client-api/6.7.0//kubernetes-client-api-6.7.0.jar +kubernetes-client/6.7.0//kubernetes-client-6.7.0.jar +kubernetes-httpclient-okhttp/6.7.0//kubernetes-httpclient-okhttp-6.7.0.jar +kubernetes-model-admissionregistration/6.7.0//kubernetes-model-admissionregistration-6.7.0.jar +kubernetes-model-apiextensions/6.7.0//kubernetes-model-apiextensions-6.7.0.jar +kubernetes-model-apps/6.7.0//kubernetes-model-apps-6.7.0.jar +kubernetes-model-autoscaling/6.7.0//kubernetes-model-autoscaling-6.7.0.jar +kubernetes-model-batch/6.7.0//kubernetes-model-batch-6.7.0.jar +kubernetes-model-certificates/6.7.0//kubernetes-model-certificates-6.7.0.jar +kubernetes-model-common/6.7.0//kubernetes-model-common-6.7.0.jar +kubernetes-model-coordination/6.7.0//kubernetes-model-coordination-6.7.0.jar +kubernetes-model-core/6.7.0//kubernetes-model-core-6.7.0.jar +kubernetes-model-discovery/6.7.0//kubernetes-model-discovery-6.7.0.jar +kubernetes-model-events/6.7.0//kubernetes-model-events-6.7.0.jar +kubernetes-model-extensions/6.7.0//kubernetes-model-extensions-6.7.0.jar +kubernetes-model-flowcontrol/6.7.0//kubernetes-model-flowcontrol-6.7.0.jar +kubernetes-model-gatewayapi/6.7.0//kubernetes-model-gatewayapi-6.7.0.jar +kubernetes-model-metrics/6.7.0//kubernetes-model-metrics-6.7.0.jar +kubernetes-model-networking/6.7.0//kubernetes-model-networking-6.7.0.jar +kubernetes-model-node/6.7.0//kubernetes-model-node-6.7.0.jar +kubernetes-model-policy/6.7.0//kubernetes-model-policy-6.7.0.jar +kubernetes-model-rbac/6.7.0//kubernetes-model-rbac-6.7.0.jar
[spark] branch master updated (c3b62708cd6 -> 18b9bd9dcb0)
This is an automated email from the ASF dual-hosted git repository. maxgekk pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/spark.git from c3b62708cd6 [SPARK-43516][ML][FOLLOW-UP] Drop vector type support in Distributed ML for spark connect add 18b9bd9dcb0 [SPARK-43945][SQL][TESTS] Fix bug for `SQLQueryTestSuite` when run on local env No new revisions were added by this update. Summary of changes: sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out | 2 +- sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43516][ML][FOLLOW-UP] Drop vector type support in Distributed ML for spark connect
This is an automated email from the ASF dual-hosted git repository. weichenxu123 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 c3b62708cd6 [SPARK-43516][ML][FOLLOW-UP] Drop vector type support in Distributed ML for spark connect c3b62708cd6 is described below commit c3b62708cd6371da08943e572a9bc0a45c1dcced Author: Weichen Xu AuthorDate: Fri Jun 2 20:28:01 2023 +0800 [SPARK-43516][ML][FOLLOW-UP] Drop vector type support in Distributed ML for spark connect ### What changes were proposed in this pull request? Drop vector type support in Distributed ML for spark connect. ### Why are the changes needed? Distributed ML is designed for supporting fitting / transforming over either spark dataframe or local pandas dataframe. Currently pandas dataframe does not have vector type similar to `spark.ml.linalg.Vector`, and Vector type does not have too much advantages except saving sparse features dataset. To make the interface consistent, we decided initial version does not support vector type. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? UT. Closes #41420 from WeichenXu123/mlv2-drop-vector-type-support. Authored-by: Weichen Xu Signed-off-by: Weichen Xu --- python/pyspark/mlv2/base.py | 6 +- python/pyspark/mlv2/tests/test_feature.py| 16 +++- python/pyspark/mlv2/tests/test_summarizer.py | 10 -- python/pyspark/mlv2/util.py | 19 --- 4 files changed, 16 insertions(+), 35 deletions(-) diff --git a/python/pyspark/mlv2/base.py b/python/pyspark/mlv2/base.py index dc503db71c0..63631eccf2f 100644 --- a/python/pyspark/mlv2/base.py +++ b/python/pyspark/mlv2/base.py @@ -134,7 +134,11 @@ class Transformer(Params, metaclass=ABCMeta): def _get_transform_fn(self) -> Callable[["pd.Series"], Any]: """ Return a transformation function that accepts an instance of `pd.Series` as input and -returns transformed result as an instance of `pd.Series` or `pd.DataFrame` +returns transformed result as an instance of `pd.Series` or `pd.DataFrame`. +If there's only one output column, the transformed result must be an +instance of `pd.Series`, if there are multiple output columns, the transformed result +must be an instance of `pd.DataFrame` with column names matching output schema +returned by `_output_columns` interface. """ raise NotImplementedError() diff --git a/python/pyspark/mlv2/tests/test_feature.py b/python/pyspark/mlv2/tests/test_feature.py index 8bc9d4c2307..eed04217a6f 100644 --- a/python/pyspark/mlv2/tests/test_feature.py +++ b/python/pyspark/mlv2/tests/test_feature.py @@ -21,8 +21,6 @@ from distutils.version import LooseVersion import numpy as np import pandas as pd -from pyspark.ml.functions import vector_to_array -from pyspark.ml.linalg import Vectors from pyspark.mlv2.feature import MaxAbsScaler, StandardScaler from pyspark.sql import SparkSession @@ -35,8 +33,8 @@ class FeatureTestsMixin: def test_max_abs_scaler(self): df1 = self.spark.createDataFrame( [ -(Vectors.dense([2.0, 3.5, 1.5]),), -(Vectors.dense([-3.0, -0.5, -2.5]),), +([2.0, 3.5, 1.5],), +([-3.0, -0.5, -2.5],), ], schema=["features"], ) @@ -49,7 +47,7 @@ class FeatureTestsMixin: np.testing.assert_allclose(list(result.scaled_features), expected_result) -local_df1 = df1.withColumn("features", vector_to_array("features")).toPandas() +local_df1 = df1.toPandas() local_fit_model = scaler.fit(local_df1) local_transform_result = local_fit_model.transform(local_df1) @@ -62,9 +60,9 @@ class FeatureTestsMixin: def test_standard_scaler(self): df1 = self.spark.createDataFrame( [ -(Vectors.dense([2.0, 3.5, 1.5]),), -(Vectors.dense([-3.0, -0.5, -2.5]),), -(Vectors.dense([1.0, -1.5, 0.5]),), +([2.0, 3.5, 1.5],), +([-3.0, -0.5, -2.5],), +([1.0, -1.5, 0.5],), ], schema=["features"], ) @@ -81,7 +79,7 @@ class FeatureTestsMixin: np.testing.assert_allclose(list(result.scaled_features), expected_result) -local_df1 = df1.withColumn("features", vector_to_array("features")).toPandas() +local_df1 = df1.toPandas() local_fit_model = scaler.fit(local_df1) local_transform_result = local_fit_model.transform(local_df1) diff --git a/python/pyspark/mlv2/tests/test_summarizer.py b/python/pyspark/mlv2/tests/test_summarizer.py index e78510b8ff4..927ef0bd
[spark-connect-go] branch master updated: [SPARK-43351][FOLLOW] Cleanup and Fix Makefile
This is an automated email from the ASF dual-hosted git repository. gurwls223 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark-connect-go.git The following commit(s) were added to refs/heads/master by this push: new 9b8c70d [SPARK-43351][FOLLOW] Cleanup and Fix Makefile 9b8c70d is described below commit 9b8c70dc949af440123de8f71302f6f8d0245253 Author: Martin Grund AuthorDate: Fri Jun 2 21:09:03 2023 +0900 [SPARK-43351][FOLLOW] Cleanup and Fix Makefile ### What changes were proposed in this pull request? Follow-up patch to fix the Makefile and make sure all tests are executed during the Github action. ### Why are the changes needed? UT / Cleanup ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT Closes #7 from grundprinzip/SPARK-43351. Authored-by: Martin Grund Signed-off-by: Hyukjin Kwon --- .github/workflows/build.yml | 2 +- .gitignore | 4 Makefile| 17 +++-- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 4b870fc..9d0c13b 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -60,5 +60,5 @@ jobs: pwd go mod download -x make gen - make check + make make test \ No newline at end of file diff --git a/.gitignore b/.gitignore index f61e556..17dc1fb 100644 --- a/.gitignore +++ b/.gitignore @@ -24,3 +24,7 @@ coverage* # Ignore IDE files .idea/ + +# Ignore binaries +cmd/spark-connect-example-raw-grpc-client/spark-connect-example-raw-grpc-client +cmd/spark-connect-example-spark-session/spark-connect-example-spark-session \ No newline at end of file diff --git a/Makefile b/Makefile index a9829da..d4d158a 100644 --- a/Makefile +++ b/Makefile @@ -37,7 +37,7 @@ GOARCH?= $(shell go version | cut -d' ' -f4 | cut -d'/' -f2) TAGS ?= netgo SHELL = bash -BINARIES := +BINARIES := cmd/spark-connect-example-spark-session cmd/spark-connect-example-raw-grpc-client # Define the location of SPARK_HOME because we need that to depend on the build paths MAKEFILE_DIR:=$(shell dirname $(realpath $(firstword $(MAKEFILE_LIST @@ -51,6 +51,16 @@ all: build build: $(BUILD_OUTPUT) $(BINARIES) internal/generated.out +cmd/spark-connect-example-raw-grpc-client: $(GOFILES_BUILD) + @echo ">> BUILD, output = $@" + @cd $@ && $(GO) build -o $(notdir $@) $(BUILDFLAGS) + @printf '%s\n' '$(OK)' + +cmd/spark-connect-example-spark-session: $(GOFILES_BUILD) + @echo ">> BUILD, output = $@" + @cd $@ && $(GO) build -o $(notdir $@) $(BUILDFLAGS) + @printf '%s\n' '$(OK)' + internal/generated.out: @echo -n ">> BUILD, output = $@" buf generate --debug -vvv @@ -66,11 +76,6 @@ $(BUILD_OUTPUT): $(GOFILES_BUILD) @$(GO) build -o $@ $(BUILDFLAGS) @printf '%s\n' '$(OK)' -check: $(ALLGOFILES) | gen - @echo -n ">> BUILD, output = $@" - @$(GO) build $(BUILDFLAGS) $^ - @printf '%s\n' '$(OK)' - lint: $(BUILD_OUTPUT) @golangci-lint run - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark-connect-go] branch master updated: [SPARK-43351] Add init Spark Connect Go files
This is an automated email from the ASF dual-hosted git repository. gurwls223 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark-connect-go.git The following commit(s) were added to refs/heads/master by this push: new 2e2d7a4 [SPARK-43351] Add init Spark Connect Go files 2e2d7a4 is described below commit 2e2d7a48f4a9626034ca11c691bcfc042d42e953 Author: Bo Yang AuthorDate: Fri Jun 2 20:15:35 2023 +0900 [SPARK-43351] Add init Spark Connect Go files ### What changes were proposed in this pull request? This pull request is to add a small Spark Connect Go client example and prototype. JIRA: https://issues.apache.org/jira/plugins/servlet/mobile#issue/SPARK-43351 ### Why are the changes needed? Spark Connect was released in Spark 3.4.0. There is no Go client yet. Better to have a Go client so Spark Connect could be used by Go programmer. ### Does this PR introduce _any_ user-facing change? Yes. User will be able to use Go to write Spark Connect application. A very simple example in Go looks like following: ``` func main() { remote := "localhost:15002" spark, _ := sql.SparkSession.Builder.Remote(remote).Build() defer spark.Stop() df, _ := spark.Sql("select 'apple' as word, 123 as count union all select 'orange' as word, 456 as count") df.Show(100, false) } ``` ### How was this patch tested? Manually tested by running the example Go code. Closes #6 from hiboyang/bo-dev-01. Lead-authored-by: Bo Yang Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .gitignore| 3 + README.md | 68 ++ client/sql/dataframe.go | 282 ++ client/sql/dataframe_test.go | 57 + client/sql/datatype.go| 63 + client/sql/plan.go| 26 ++ client/sql/row.go | 35 +++ client/sql/sparksession.go| 134 ++ client/sql/structtype.go | 28 +++ cmd/spark-connect-example-raw-grpc-client/main.go | 64 + cmd/spark-connect-example-spark-session/main.go | 73 ++ go.sum| 31 +++ 12 files changed, 864 insertions(+) diff --git a/.gitignore b/.gitignore index 0bca2cf..f61e556 100644 --- a/.gitignore +++ b/.gitignore @@ -21,3 +21,6 @@ internal/generated.out # Ignore Coverage Files coverage* + +# Ignore IDE files +.idea/ diff --git a/README.md b/README.md index b2be4d3..4422b9a 100644 --- a/README.md +++ b/README.md @@ -21,6 +21,74 @@ git submodule init --depth 1 make gen && make check && make test ``` +## Spark Connect Go Application Example + +A very simple example in Go looks like following: + +``` +func main() { + remote := "localhost:15002" + spark, _ := sql.SparkSession.Builder.Remote(remote).Build() + defer spark.Stop() + + df, _ := spark.Sql("select 'apple' as word, 123 as count union all select 'orange' as word, 456 as count") + df.Show(100, false) +} +``` + +## High Level Design + +Following [diagram](https://textik.com/#ac299c8f32c4c342) shows main code in current prototype: + +``` ++---+ +| | +| dataFrameImpl | +| | ++---+ + | + | + + ++---+ +| | +| sparkSessionImpl | +| | ++---+ + | + |
[spark] branch branch-3.4 updated: [SPARK-43949][PYTHON] Upgrade cloudpickle to 2.2.1
This is an automated email from the ASF dual-hosted git repository. gurwls223 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 746c906cce7 [SPARK-43949][PYTHON] Upgrade cloudpickle to 2.2.1 746c906cce7 is described below commit 746c906cce76992ad531a1b6b57e05470085c917 Author: Hyukjin Kwon AuthorDate: Fri Jun 2 19:38:13 2023 +0900 [SPARK-43949][PYTHON] Upgrade cloudpickle to 2.2.1 This PR proposes to upgrade Cloudpickle from 2.2.0 to 2.2.1. Cloudpickle 2.2.1 has a fix (https://github.com/cloudpipe/cloudpickle/pull/495) for namedtuple issue (https://github.com/cloudpipe/cloudpickle/issues/460). PySpark relies on namedtuple heavily especially for RDD. We should upgrade and fix it. Yes, see https://github.com/cloudpipe/cloudpickle/issues/460. Relies on cloudpickle's unittests. Existing test cases should pass too. Closes #41433 from HyukjinKwon/cloudpickle-upgrade. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 085dfeb2bed61f6d43d9b99b299373e797ac8f17) Signed-off-by: Hyukjin Kwon --- python/pyspark/cloudpickle/__init__.py | 2 +- python/pyspark/cloudpickle/cloudpickle_fast.py | 4 ++-- python/pyspark/cloudpickle/compat.py | 7 ++- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/python/pyspark/cloudpickle/__init__.py b/python/pyspark/cloudpickle/__init__.py index efbf1178d43..af35a0a194b 100644 --- a/python/pyspark/cloudpickle/__init__.py +++ b/python/pyspark/cloudpickle/__init__.py @@ -5,4 +5,4 @@ from pyspark.cloudpickle.cloudpickle_fast import CloudPickler, dumps, dump # no # expose their Pickler subclass at top-level under the "Pickler" name. Pickler = CloudPickler -__version__ = '2.2.0' +__version__ = '2.2.1' diff --git a/python/pyspark/cloudpickle/cloudpickle_fast.py b/python/pyspark/cloudpickle/cloudpickle_fast.py index 8741dcbdaaa..63aaffa096b 100644 --- a/python/pyspark/cloudpickle/cloudpickle_fast.py +++ b/python/pyspark/cloudpickle/cloudpickle_fast.py @@ -111,8 +111,8 @@ load, loads = pickle.load, pickle.loads def _class_getnewargs(obj): type_kwargs = {} -if "__slots__" in obj.__dict__: -type_kwargs["__slots__"] = obj.__slots__ +if "__module__" in obj.__dict__: +type_kwargs["__module__"] = obj.__module__ __dict__ = obj.__dict__.get('__dict__', None) if isinstance(__dict__, property): diff --git a/python/pyspark/cloudpickle/compat.py b/python/pyspark/cloudpickle/compat.py index afa285f6290..5e9b52773d2 100644 --- a/python/pyspark/cloudpickle/compat.py +++ b/python/pyspark/cloudpickle/compat.py @@ -7,7 +7,12 @@ if sys.version_info < (3, 8): from pickle5 import Pickler # noqa: F401 except ImportError: import pickle # noqa: F401 + +# Use the Python pickler for old CPython versions from pickle import _Pickler as Pickler # noqa: F401 else: import pickle # noqa: F401 -from _pickle import Pickler # noqa: F401 + +# Pickler will the C implementation in CPython and the Python +# implementation in PyPy +from pickle import Pickler # noqa: F401 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43949][PYTHON] Upgrade cloudpickle to 2.2.1
This is an automated email from the ASF dual-hosted git repository. gurwls223 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 085dfeb2bed [SPARK-43949][PYTHON] Upgrade cloudpickle to 2.2.1 085dfeb2bed is described below commit 085dfeb2bed61f6d43d9b99b299373e797ac8f17 Author: Hyukjin Kwon AuthorDate: Fri Jun 2 19:38:13 2023 +0900 [SPARK-43949][PYTHON] Upgrade cloudpickle to 2.2.1 ### What changes were proposed in this pull request? This PR proposes to upgrade Cloudpickle from 2.2.0 to 2.2.1. ### Why are the changes needed? Cloudpickle 2.2.1 has a fix (https://github.com/cloudpipe/cloudpickle/pull/495) for namedtuple issue (https://github.com/cloudpipe/cloudpickle/issues/460). PySpark relies on namedtuple heavily especially for RDD. We should upgrade and fix it. ### Does this PR introduce _any_ user-facing change? Yes, see https://github.com/cloudpipe/cloudpickle/issues/460. ### How was this patch tested? Relies on cloudpickle's unittests. Existing test cases should pass too. Closes #41433 from HyukjinKwon/cloudpickle-upgrade. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/cloudpickle/__init__.py | 2 +- python/pyspark/cloudpickle/cloudpickle_fast.py | 4 ++-- python/pyspark/cloudpickle/compat.py | 17 +++-- 3 files changed, 18 insertions(+), 5 deletions(-) diff --git a/python/pyspark/cloudpickle/__init__.py b/python/pyspark/cloudpickle/__init__.py index efbf1178d43..af35a0a194b 100644 --- a/python/pyspark/cloudpickle/__init__.py +++ b/python/pyspark/cloudpickle/__init__.py @@ -5,4 +5,4 @@ from pyspark.cloudpickle.cloudpickle_fast import CloudPickler, dumps, dump # no # expose their Pickler subclass at top-level under the "Pickler" name. Pickler = CloudPickler -__version__ = '2.2.0' +__version__ = '2.2.1' diff --git a/python/pyspark/cloudpickle/cloudpickle_fast.py b/python/pyspark/cloudpickle/cloudpickle_fast.py index 8741dcbdaaa..63aaffa096b 100644 --- a/python/pyspark/cloudpickle/cloudpickle_fast.py +++ b/python/pyspark/cloudpickle/cloudpickle_fast.py @@ -111,8 +111,8 @@ load, loads = pickle.load, pickle.loads def _class_getnewargs(obj): type_kwargs = {} -if "__slots__" in obj.__dict__: -type_kwargs["__slots__"] = obj.__slots__ +if "__module__" in obj.__dict__: +type_kwargs["__module__"] = obj.__module__ __dict__ = obj.__dict__.get('__dict__', None) if isinstance(__dict__, property): diff --git a/python/pyspark/cloudpickle/compat.py b/python/pyspark/cloudpickle/compat.py index 837d0f279ab..5e9b52773d2 100644 --- a/python/pyspark/cloudpickle/compat.py +++ b/python/pyspark/cloudpickle/compat.py @@ -1,5 +1,18 @@ import sys -import pickle # noqa: F401 -from pickle import Pickler # noqa: F401 +if sys.version_info < (3, 8): +try: +import pickle5 as pickle # noqa: F401 +from pickle5 import Pickler # noqa: F401 +except ImportError: +import pickle # noqa: F401 + +# Use the Python pickler for old CPython versions +from pickle import _Pickler as Pickler # noqa: F401 +else: +import pickle # noqa: F401 + +# Pickler will the C implementation in CPython and the Python +# implementation in PyPy +from pickle import Pickler # noqa: F401 - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
[spark] branch master updated: [SPARK-43063][SQL][FOLLOWUP] Add a space between `->` and value
This is an automated email from the ASF dual-hosted git repository. gurwls223 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 fed51b9eeaf [SPARK-43063][SQL][FOLLOWUP] Add a space between `->` and value fed51b9eeaf is described below commit fed51b9eeaf50d0b2cbc9e14b91b6dc3fc660075 Author: Dongjoon Hyun AuthorDate: Fri Jun 2 19:34:30 2023 +0900 [SPARK-43063][SQL][FOLLOWUP] Add a space between `->` and value ### What changes were proposed in this pull request? This is a follow-up of #40922. This PR aims to add a space between `->` and value. It seems to be missed here because the original PR already have the same code pattern in other place. https://github.com/apache/spark/blob/74b04eeffdc4765f56fe3a9e97165b15ed4e2c73/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala#L114 ### Why are the changes needed? **BEFORE** ``` scala> sql("select map('k', null)").show() ++ |map(k, NULL)| ++ | {k ->NULL}| ++ ``` **AFTER** ``` scala> sql("select map('k', null)").show() ++ |map(k, NULL)| ++ | {k -> NULL}| ++ ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual review. Closes #41432 from dongjoon-hyun/SPARK-43063. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala index 7304d6739e8..ab9f451e88f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ToStringBase.scala @@ -100,7 +100,7 @@ trait ToStringBase { self: UnaryExpression with TimeZoneAwareExpression => builder.append(keyToUTF8String(keyArray.get(0, kt)).asInstanceOf[UTF8String]) builder.append(" ->") if (valueArray.isNullAt(0)) { -if (nullString.nonEmpty) builder.append(nullString) +if (nullString.nonEmpty) builder.append(" " + nullString) } else { builder.append(" ") builder.append(valueToUTF8String(valueArray.get(0, vt)).asInstanceOf[UTF8String]) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org