You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by jo...@apache.org on 2015/10/22 00:07:20 UTC

spark git commit: [SPARK-10151][SQL] Support invocation of hive macro

Repository: spark
Updated Branches:
  refs/heads/master dce2f8c9d -> f481090a7


[SPARK-10151][SQL] Support invocation of hive macro

Macro in hive (which is GenericUDFMacro) contains real function inside of it but it's not conveyed to tasks, resulting null-pointer exception.

Author: navis.ryu <na...@apache.org>

Closes #8354 from navis/SPARK-10151.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f481090a
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f481090a
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f481090a

Branch: refs/heads/master
Commit: f481090a71940f06602a73f5bbd004980dea026f
Parents: dce2f8c
Author: navis.ryu <na...@apache.org>
Authored: Wed Oct 21 15:07:08 2015 -0700
Committer: Josh Rosen <jo...@databricks.com>
Committed: Wed Oct 21 15:07:08 2015 -0700

----------------------------------------------------------------------
 .../spark/sql/hive/execution/HiveCompatibilitySuite.scala   | 1 +
 .../main/scala/org/apache/spark/sql/hive/HiveContext.scala  | 9 +++++++--
 .../scala/org/apache/spark/sql/hive/HiveInspectors.scala    | 4 ++++
 .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala   | 2 ++
 .../src/main/scala/org/apache/spark/sql/hive/HiveShim.scala | 9 ++++-----
 .../src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala | 5 ++++-
 .../golden/macro-0-50131c0ba7b7a6b65c789a5a8497bada         | 1 +
 .../golden/macro-1-5ff5e8795c13303db5d3ea88e1e918b6         | 0
 .../golden/macro-10-45148a37f6ee9cf498dc7308cbd81a1c        | 0
 .../golden/macro-11-f55b8684c77f6eefc2618ba79e5e0587        | 1 +
 .../golden/macro-12-62b999122975c2a5de8e49fee089c041        | 0
 .../golden/macro-13-87e53d2b4c84098e662779e8f0a59084        | 1 +
 .../golden/macro-14-3a31df84432674ad410f44b137e32c2d        | 0
 .../golden/macro-15-56966c45104c0d9bc407e79538c2c029        | 0
 .../golden/macro-16-56966c45104c0d9bc407e79538c2c029        | 0
 .../golden/macro-17-b3864f1d19fdb88b3b74f6d74a0ba548        | 1 +
 .../golden/macro-18-bddb2fe17cd4d850c4462b7eb2b9bc2a        | 0
 .../golden/macro-19-e3c828c372607b8bf7be00a99359b662        | 0
 .../golden/macro-2-fde44c7854a9897acb4c2f78f24c8eec         | 1 +
 .../golden/macro-20-cb252a243d59809930a4ff371cbfa292        | 0
 .../golden/macro-21-cb252a243d59809930a4ff371cbfa292        | 0
 .../golden/macro-3-ddc4cb920b0a68e06551cd34ae4e29ff         | 0
 .../golden/macro-4-86292bbb7f147393c38bca051768dbda         | 0
 .../golden/macro-5-ca270bff813e5ab18a6a799016693aa8         | 0
 .../golden/macro-6-8976be22af3aba0cc4905e014b4e24fe         | 0
 .../golden/macro-7-decde0a59183a393e580941c633d3c5c         | 1 +
 .../golden/macro-8-3d25ffda9ab348f3e39ad967fc0e5020         | 0
 .../golden/macro-9-db5f5172704da1e6dd5d59c136b83e7e         | 0
 28 files changed, 28 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 8f29fa9..eed9e43 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -684,6 +684,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "load_file_with_space_in_the_name",
     "loadpart1",
     "louter_join_ppr",
+    "macro",
     "mapjoin_distinct",
     "mapjoin_filter_on_outerjoin",
     "mapjoin_mapjoin",

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index c7460d4..c328734 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -21,6 +21,7 @@ import java.io.File
 import java.net.{URL, URLClassLoader}
 import java.sql.Timestamp
 import java.util.concurrent.TimeUnit
+import java.util.regex.Pattern
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.HashMap
@@ -592,10 +593,14 @@ class HiveContext private[hive](
     )
   }
 
+  private def functionOrMacroDDLPattern(command: String) = Pattern.compile(
+    ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL).matcher(command)
+
   protected[hive] def runSqlHive(sql: String): Seq[String] = {
-    if (sql.toLowerCase.contains("create temporary function")) {
+    val command = sql.trim.toLowerCase
+    if (functionOrMacroDDLPattern(command).matches()) {
       executionHive.runSqlHive(sql)
-    } else if (sql.trim.toLowerCase.startsWith("set")) {
+    } else if (command.startsWith("set")) {
       metadataHive.runSqlHive(sql)
       executionHive.runSqlHive(sql)
     } else {

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index cfe2bb0..43c238f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -723,6 +723,10 @@ private[hive] trait HiveInspectors {
         inspectorToDataType(m.getMapValueObjectInspector))
     case _: WritableStringObjectInspector => StringType
     case _: JavaStringObjectInspector => StringType
+    case _: WritableHiveVarcharObjectInspector => StringType
+    case _: JavaHiveVarcharObjectInspector => StringType
+    case _: WritableHiveCharObjectInspector => StringType
+    case _: JavaHiveCharObjectInspector => StringType
     case _: WritableIntObjectInspector => IntegerType
     case _: JavaIntObjectInspector => IntegerType
     case _: WritableDoubleObjectInspector => DoubleType

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index d4ff5cc..3697761 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -117,6 +117,7 @@ private[hive] object HiveQl extends Logging {
     "TOK_CREATEDATABASE",
     "TOK_CREATEFUNCTION",
     "TOK_CREATEINDEX",
+    "TOK_CREATEMACRO",
     "TOK_CREATEROLE",
 
     "TOK_DESCDATABASE",
@@ -125,6 +126,7 @@ private[hive] object HiveQl extends Logging {
     "TOK_DROPDATABASE",
     "TOK_DROPFUNCTION",
     "TOK_DROPINDEX",
+    "TOK_DROPMACRO",
     "TOK_DROPROLE",
     "TOK_DROPTABLE_PROPERTIES",
     "TOK_DROPVIEW",

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
index 004805f..f069761 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
@@ -117,9 +117,10 @@ private[hive] object HiveShim {
    * Detail discussion can be found at https://github.com/apache/spark/pull/3640
    *
    * @param functionClassName UDF class name
+   * @param instance optional UDF instance which contains additional information (for macro)
    */
-  private[hive] case class HiveFunctionWrapper(var functionClassName: String)
-    extends java.io.Externalizable {
+  private[hive] case class HiveFunctionWrapper(var functionClassName: String,
+    private var instance: AnyRef = null) extends java.io.Externalizable {
 
     // for Serialization
     def this() = this(null)
@@ -154,8 +155,6 @@ private[hive] object HiveShim {
       serializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), function, out)
     }
 
-    private var instance: AnyRef = null
-
     def writeExternal(out: java.io.ObjectOutput) {
       // output the function name
       out.writeUTF(functionClassName)
@@ -184,7 +183,7 @@ private[hive] object HiveShim {
         // read the function in bytes
         val functionInBytesLength = in.readInt()
         val functionInBytes = new Array[Byte](functionInBytesLength)
-        in.read(functionInBytes, 0, functionInBytesLength)
+        in.readFully(functionInBytes)
 
         // deserialize the function object via Hive Utilities
         instance = deserializePlan[AnyRef](new java.io.ByteArrayInputStream(functionInBytes),

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
index a2ebf65..f57b206 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala
@@ -64,7 +64,10 @@ private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry)
       // don't satisfy the hive UDF, such as type mismatch, input number mismatch, etc. Here we
       // catch the exception and throw AnalysisException instead.
       try {
-        if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) {
+        if (classOf[GenericUDFMacro].isAssignableFrom(functionInfo.getFunctionClass)) {
+          HiveGenericUDF(
+            new HiveFunctionWrapper(functionClassName, functionInfo.getGenericUDF), children)
+        } else if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) {
           HiveSimpleUDF(new HiveFunctionWrapper(functionClassName), children)
         } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) {
           HiveGenericUDF(new HiveFunctionWrapper(functionClassName), children)

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-0-50131c0ba7b7a6b65c789a5a8497bada
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/macro-0-50131c0ba7b7a6b65c789a5a8497bada
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/macro-0-50131c0ba7b7a6b65c789a5a8497bada
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-1-5ff5e8795c13303db5d3ea88e1e918b6
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-1-5ff5e8795c13303db5d3ea88e1e918b6 b/sql/hive/src/test/resources/golden/macro-1-5ff5e8795c13303db5d3ea88e1e918b6
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-10-45148a37f6ee9cf498dc7308cbd81a1c
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-10-45148a37f6ee9cf498dc7308cbd81a1c b/sql/hive/src/test/resources/golden/macro-10-45148a37f6ee9cf498dc7308cbd81a1c
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-11-f55b8684c77f6eefc2618ba79e5e0587
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-11-f55b8684c77f6eefc2618ba79e5e0587 b/sql/hive/src/test/resources/golden/macro-11-f55b8684c77f6eefc2618ba79e5e0587
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/macro-11-f55b8684c77f6eefc2618ba79e5e0587
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-12-62b999122975c2a5de8e49fee089c041
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-12-62b999122975c2a5de8e49fee089c041 b/sql/hive/src/test/resources/golden/macro-12-62b999122975c2a5de8e49fee089c041
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-13-87e53d2b4c84098e662779e8f0a59084
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-13-87e53d2b4c84098e662779e8f0a59084 b/sql/hive/src/test/resources/golden/macro-13-87e53d2b4c84098e662779e8f0a59084
new file mode 100644
index 0000000..d00491f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/macro-13-87e53d2b4c84098e662779e8f0a59084
@@ -0,0 +1 @@
+1

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-14-3a31df84432674ad410f44b137e32c2d
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-14-3a31df84432674ad410f44b137e32c2d b/sql/hive/src/test/resources/golden/macro-14-3a31df84432674ad410f44b137e32c2d
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-15-56966c45104c0d9bc407e79538c2c029
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-15-56966c45104c0d9bc407e79538c2c029 b/sql/hive/src/test/resources/golden/macro-15-56966c45104c0d9bc407e79538c2c029
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-16-56966c45104c0d9bc407e79538c2c029
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-16-56966c45104c0d9bc407e79538c2c029 b/sql/hive/src/test/resources/golden/macro-16-56966c45104c0d9bc407e79538c2c029
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-17-b3864f1d19fdb88b3b74f6d74a0ba548
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-17-b3864f1d19fdb88b3b74f6d74a0ba548 b/sql/hive/src/test/resources/golden/macro-17-b3864f1d19fdb88b3b74f6d74a0ba548
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/macro-17-b3864f1d19fdb88b3b74f6d74a0ba548
@@ -0,0 +1 @@
+10

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-18-bddb2fe17cd4d850c4462b7eb2b9bc2a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-18-bddb2fe17cd4d850c4462b7eb2b9bc2a b/sql/hive/src/test/resources/golden/macro-18-bddb2fe17cd4d850c4462b7eb2b9bc2a
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-19-e3c828c372607b8bf7be00a99359b662
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-19-e3c828c372607b8bf7be00a99359b662 b/sql/hive/src/test/resources/golden/macro-19-e3c828c372607b8bf7be00a99359b662
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-2-fde44c7854a9897acb4c2f78f24c8eec
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-2-fde44c7854a9897acb4c2f78f24c8eec b/sql/hive/src/test/resources/golden/macro-2-fde44c7854a9897acb4c2f78f24c8eec
new file mode 100644
index 0000000..b49805f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/macro-2-fde44c7854a9897acb4c2f78f24c8eec
@@ -0,0 +1 @@
+0.8807970779778823

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-20-cb252a243d59809930a4ff371cbfa292
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-20-cb252a243d59809930a4ff371cbfa292 b/sql/hive/src/test/resources/golden/macro-20-cb252a243d59809930a4ff371cbfa292
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-21-cb252a243d59809930a4ff371cbfa292
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-21-cb252a243d59809930a4ff371cbfa292 b/sql/hive/src/test/resources/golden/macro-21-cb252a243d59809930a4ff371cbfa292
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-3-ddc4cb920b0a68e06551cd34ae4e29ff
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-3-ddc4cb920b0a68e06551cd34ae4e29ff b/sql/hive/src/test/resources/golden/macro-3-ddc4cb920b0a68e06551cd34ae4e29ff
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-4-86292bbb7f147393c38bca051768dbda
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-4-86292bbb7f147393c38bca051768dbda b/sql/hive/src/test/resources/golden/macro-4-86292bbb7f147393c38bca051768dbda
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-5-ca270bff813e5ab18a6a799016693aa8
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-5-ca270bff813e5ab18a6a799016693aa8 b/sql/hive/src/test/resources/golden/macro-5-ca270bff813e5ab18a6a799016693aa8
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-6-8976be22af3aba0cc4905e014b4e24fe
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-6-8976be22af3aba0cc4905e014b4e24fe b/sql/hive/src/test/resources/golden/macro-6-8976be22af3aba0cc4905e014b4e24fe
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-7-decde0a59183a393e580941c633d3c5c
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-7-decde0a59183a393e580941c633d3c5c b/sql/hive/src/test/resources/golden/macro-7-decde0a59183a393e580941c633d3c5c
new file mode 100644
index 0000000..0cfbf08
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/macro-7-decde0a59183a393e580941c633d3c5c
@@ -0,0 +1 @@
+2

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-8-3d25ffda9ab348f3e39ad967fc0e5020
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-8-3d25ffda9ab348f3e39ad967fc0e5020 b/sql/hive/src/test/resources/golden/macro-8-3d25ffda9ab348f3e39ad967fc0e5020
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f481090a/sql/hive/src/test/resources/golden/macro-9-db5f5172704da1e6dd5d59c136b83e7e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/macro-9-db5f5172704da1e6dd5d59c136b83e7e b/sql/hive/src/test/resources/golden/macro-9-db5f5172704da1e6dd5d59c136b83e7e
new file mode 100644
index 0000000..e69de29


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