-
Notifications
You must be signed in to change notification settings - Fork 28.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-35380][SQL] Loading SparkSessionExtensions from ServiceLoader
### What changes were proposed in this pull request? In yaooqinn/itachi#8, we had a discussion about the current extension injection for the spark session. We've agreed that the current way is not that convenient for both third-party developers and end-users. It's much simple if third-party developers can provide a resource file that contains default extensions for Spark to load ahead ### Why are the changes needed? better use experience ### Does this PR introduce _any_ user-facing change? no, dev change ### How was this patch tested? new tests Closes #32515 from yaooqinn/SPARK-35380. Authored-by: Kent Yao <[email protected]> Signed-off-by: Kent Yao <[email protected]>
- Loading branch information
Showing
10 changed files
with
276 additions
and
5 deletions.
There are no files selected for viewing
1 change: 1 addition & 0 deletions
1
.../src/main/resources/META-INF/services/org.apache.spark.sql.SparkSessionExtensionsProvider
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1 @@ | ||
org.apache.spark.examples.extensions.SessionExtensionsWithLoader |
31 changes: 31 additions & 0 deletions
31
examples/src/main/scala/org/apache/spark/examples/extensions/AgeExample.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
/* | ||
* 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.examples.extensions | ||
|
||
import org.apache.spark.sql.catalyst.expressions.{CurrentDate, Expression, RuntimeReplaceable, SubtractDates} | ||
|
||
/** | ||
* How old are you in days? | ||
*/ | ||
case class AgeExample(birthday: Expression, child: Expression) extends RuntimeReplaceable { | ||
|
||
def this(birthday: Expression) = this(birthday, SubtractDates(CurrentDate(), birthday)) | ||
override def exprsReplaced: Seq[Expression] = Seq(birthday) | ||
|
||
override protected def withNewChildInternal(newChild: Expression): Expression = copy(newChild) | ||
} |
31 changes: 31 additions & 0 deletions
31
...les/src/main/scala/org/apache/spark/examples/extensions/SessionExtensionsWithLoader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
/* | ||
* 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.examples.extensions | ||
|
||
import org.apache.spark.sql.{SparkSessionExtensions, SparkSessionExtensionsProvider} | ||
import org.apache.spark.sql.catalyst.FunctionIdentifier | ||
import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} | ||
|
||
class SessionExtensionsWithLoader extends SparkSessionExtensionsProvider { | ||
override def apply(v1: SparkSessionExtensions): Unit = { | ||
v1.injectFunction( | ||
(new FunctionIdentifier("age_two"), | ||
new ExpressionInfo(classOf[AgeExample].getName, | ||
"age_two"), (children: Seq[Expression]) => new AgeExample(children.head))) | ||
} | ||
} |
31 changes: 31 additions & 0 deletions
31
.../src/main/scala/org/apache/spark/examples/extensions/SessionExtensionsWithoutLoader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
/* | ||
* 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.examples.extensions | ||
|
||
import org.apache.spark.sql.{SparkSessionExtensions, SparkSessionExtensionsProvider} | ||
import org.apache.spark.sql.catalyst.FunctionIdentifier | ||
import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} | ||
|
||
class SessionExtensionsWithoutLoader extends SparkSessionExtensionsProvider { | ||
override def apply(v1: SparkSessionExtensions): Unit = { | ||
v1.injectFunction( | ||
(new FunctionIdentifier("age_one"), | ||
new ExpressionInfo(classOf[AgeExample].getName, | ||
"age_one"), (children: Seq[Expression]) => new AgeExample(children.head))) | ||
} | ||
} |
38 changes: 38 additions & 0 deletions
38
...ples/src/main/scala/org/apache/spark/examples/extensions/SparkSessionExtensionsTest.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,38 @@ | ||
/* | ||
* 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.examples.extensions | ||
|
||
import org.apache.spark.sql.SparkSession | ||
|
||
/** | ||
* [[SessionExtensionsWithLoader]] is registered in | ||
* src/main/resources/META-INF/services/org.apache.spark.sql.SparkSessionExtensionsProvider | ||
* | ||
* [[SessionExtensionsWithoutLoader]] is registered via spark.sql.extensions | ||
*/ | ||
object SparkSessionExtensionsTest { | ||
|
||
def main(args: Array[String]): Unit = { | ||
val spark = SparkSession | ||
.builder() | ||
.appName("SparkSessionExtensionsTest") | ||
.config("spark.sql.extensions", classOf[SessionExtensionsWithoutLoader].getName) | ||
.getOrCreate() | ||
spark.sql("SELECT age_one('2018-11-17'), age_two('2018-11-17')").show() | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
82 changes: 82 additions & 0 deletions
82
sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensionsProvider.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,82 @@ | ||
/* | ||
* 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 org.apache.spark.annotation.{DeveloperApi, Since, Unstable} | ||
|
||
// scalastyle:off line.size.limit | ||
/** | ||
* :: Unstable :: | ||
* | ||
* Base trait for implementations used by [[SparkSessionExtensions]] | ||
* | ||
* | ||
* For example, now we have an external function named `Age` to register as an extension for SparkSession: | ||
* | ||
* | ||
* {{{ | ||
* package org.apache.spark.examples.extensions | ||
* | ||
* import org.apache.spark.sql.catalyst.expressions.{CurrentDate, Expression, RuntimeReplaceable, SubtractDates} | ||
* | ||
* case class Age(birthday: Expression, child: Expression) extends RuntimeReplaceable { | ||
* | ||
* def this(birthday: Expression) = this(birthday, SubtractDates(CurrentDate(), birthday)) | ||
* override def exprsReplaced: Seq[Expression] = Seq(birthday) | ||
* override protected def withNewChildInternal(newChild: Expression): Expression = copy(newChild) | ||
* } | ||
* }}} | ||
* | ||
* We need to create our extension which inherits [[SparkSessionExtensionsProvider]] | ||
* Example: | ||
* | ||
* {{{ | ||
* package org.apache.spark.examples.extensions | ||
* | ||
* import org.apache.spark.sql.{SparkSessionExtensions, SparkSessionExtensionsProvider} | ||
* import org.apache.spark.sql.catalyst.FunctionIdentifier | ||
* import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} | ||
* | ||
* class MyExtensions extends SparkSessionExtensionsProvider { | ||
* override def apply(v1: SparkSessionExtensions): Unit = { | ||
* v1.injectFunction( | ||
* (new FunctionIdentifier("age"), | ||
* new ExpressionInfo(classOf[Age].getName, "age"), | ||
* (children: Seq[Expression]) => new Age(children.head))) | ||
* } | ||
* } | ||
* }}} | ||
* | ||
* Then, we can inject `MyExtensions` in three ways, | ||
* <ul> | ||
* <li>withExtensions of [[SparkSession.Builder]]</li> | ||
* <li>Config - spark.sql.extensions</li> | ||
* <li>[[java.util.ServiceLoader]] - Add to src/main/resources/META-INF/services/org.apache.spark.sql.SparkSessionExtensionsProvider</li> | ||
* </ul> | ||
* | ||
* @see [[SparkSessionExtensions]] | ||
* @see [[SparkSession.Builder]] | ||
* @see [[java.util.ServiceLoader]] | ||
* | ||
* @since 3.2.0 | ||
*/ | ||
@DeveloperApi | ||
@Unstable | ||
@Since("3.2.0") | ||
trait SparkSessionExtensionsProvider extends Function1[SparkSessionExtensions, Unit] | ||
// scalastyle:on line.size.limit |
1 change: 1 addition & 0 deletions
1
.../src/test/resources/META-INF/services/org.apache.spark.sql.SparkSessionExtensionsProvider
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1 @@ | ||
org.apache.spark.sql.YourExtensions |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters