You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2018/09/06 04:36:05 UTC
spark git commit: [SPARK-25252][SQL] Support arrays of any types by
to_json
Repository: spark
Updated Branches:
refs/heads/master 0a5a49a51 -> d749d034a
[SPARK-25252][SQL] Support arrays of any types by to_json
## What changes were proposed in this pull request?
In the PR, I propose to extended `to_json` and support any types as element types of input arrays. It should allow converting arrays of primitive types and arrays of arrays. For example:
```
select to_json(array('1','2','3'))
> ["1","2","3"]
select to_json(array(array(1,2,3),array(4)))
> [[1,2,3],[4]]
```
## How was this patch tested?
Added a couple sql tests for arrays of primitive type and of arrays. Also I added round trip test `from_json` -> `to_json`.
Closes #22226 from MaxGekk/to_json-array.
Authored-by: Maxim Gekk <ma...@databricks.com>
Signed-off-by: hyukjinkwon <gu...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d749d034
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d749d034
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d749d034
Branch: refs/heads/master
Commit: d749d034a80f528932f613ac97f13cfb99acd207
Parents: 0a5a49a
Author: Maxim Gekk <ma...@databricks.com>
Authored: Thu Sep 6 12:35:59 2018 +0800
Committer: hyukjinkwon <gu...@apache.org>
Committed: Thu Sep 6 12:35:59 2018 +0800
----------------------------------------------------------------------
R/pkg/R/functions.R | 4 +-
R/pkg/tests/fulltests/test_sparkSQL.R | 9 ++++
python/pyspark/sql/functions.py | 12 +++--
.../catalyst/expressions/jsonExpressions.scala | 40 ++++++++--------
.../sql/catalyst/json/JacksonGenerator.scala | 29 ++++++------
.../spark/sql/catalyst/json/JacksonUtils.scala | 12 +++--
.../scala/org/apache/spark/sql/functions.scala | 18 +++----
.../sql-tests/inputs/json-functions.sql | 5 ++
.../sql-tests/results/json-functions.sql.out | 22 +++++++--
.../apache/spark/sql/JsonFunctionsSuite.scala | 49 ++++++++++++++++++++
10 files changed, 139 insertions(+), 61 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/R/pkg/R/functions.R
----------------------------------------------------------------------
diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R
index d157acc..572dee5 100644
--- a/R/pkg/R/functions.R
+++ b/R/pkg/R/functions.R
@@ -1699,8 +1699,8 @@ setMethod("to_date",
})
#' @details
-#' \code{to_json}: Converts a column containing a \code{structType}, array of \code{structType},
-#' a \code{mapType} or array of \code{mapType} into a Column of JSON string.
+#' \code{to_json}: Converts a column containing a \code{structType}, a \code{mapType}
+#' or an \code{arrayType} into a Column of JSON string.
#' Resolving the Column can fail if an unsupported type is encountered.
#'
#' @rdname column_collection_functions
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/R/pkg/tests/fulltests/test_sparkSQL.R
----------------------------------------------------------------------
diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R
index 5c07a02..0c4bdb3 100644
--- a/R/pkg/tests/fulltests/test_sparkSQL.R
+++ b/R/pkg/tests/fulltests/test_sparkSQL.R
@@ -1686,6 +1686,15 @@ test_that("column functions", {
expect_true(any(apply(s, 1, function(x) { x[[1]]$age == 16 })))
}
+ # Test to_json() supports arrays of primitive types and arrays
+ df <- sql("SELECT array(19, 42, 70) as age")
+ j <- collect(select(df, alias(to_json(df$age), "json")))
+ expect_equal(j[order(j$json), ][1], "[19,42,70]")
+
+ df <- sql("SELECT array(array(1, 2), array(3, 4)) as matrix")
+ j <- collect(select(df, alias(to_json(df$matrix), "json")))
+ expect_equal(j[order(j$json), ][1], "[[1,2],[3,4]]")
+
# passing option
df <- as.DataFrame(list(list("col" = "{\"date\":\"21/10/2014\"}")))
schema2 <- structType(structField("date", "date"))
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/python/pyspark/sql/functions.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index d58d8d1..864780e 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -2289,13 +2289,11 @@ def from_json(col, schema, options={}):
@since(2.1)
def to_json(col, options={}):
"""
- Converts a column containing a :class:`StructType`, :class:`ArrayType` of
- :class:`StructType`\\s, a :class:`MapType` or :class:`ArrayType` of :class:`MapType`\\s
+ Converts a column containing a :class:`StructType`, :class:`ArrayType` or a :class:`MapType`
into a JSON string. Throws an exception, in the case of an unsupported type.
- :param col: name of column containing the struct, array of the structs, the map or
- array of the maps.
- :param options: options to control converting. accepts the same options as the json datasource
+ :param col: name of column containing a struct, an array or a map.
+ :param options: options to control converting. accepts the same options as the JSON datasource
>>> from pyspark.sql import Row
>>> from pyspark.sql.types import *
@@ -2315,6 +2313,10 @@ def to_json(col, options={}):
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
+ >>> data = [(1, ["Alice", "Bob"])]
+ >>> df = spark.createDataFrame(data, ("key", "value"))
+ >>> df.select(to_json(df.value).alias("json")).collect()
+ [Row(json=u'["Alice","Bob"]')]
"""
sc = SparkContext._active_spark_context
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
index 11cc887..bd9090a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
@@ -613,12 +613,11 @@ case class JsonToStructs(
}
/**
- * Converts a [[StructType]], [[ArrayType]] of [[StructType]]s, [[MapType]]
- * or [[ArrayType]] of [[MapType]]s to a json output string.
+ * Converts a [[StructType]], [[ArrayType]] or [[MapType]] to a JSON output string.
*/
// scalastyle:off line.size.limit
@ExpressionDescription(
- usage = "_FUNC_(expr[, options]) - Returns a json string with a given struct value",
+ usage = "_FUNC_(expr[, options]) - Returns a JSON string with a given struct value",
examples = """
Examples:
> SELECT _FUNC_(named_struct('a', 1, 'b', 2));
@@ -660,15 +659,10 @@ case class StructsToJson(
@transient
lazy val gen = new JacksonGenerator(
- rowSchema, writer, new JSONOptions(options, timeZoneId.get))
+ inputSchema, writer, new JSONOptions(options, timeZoneId.get))
@transient
- lazy val rowSchema = child.dataType match {
- case st: StructType => st
- case ArrayType(st: StructType, _) => st
- case mt: MapType => mt
- case ArrayType(mt: MapType, _) => mt
- }
+ lazy val inputSchema = child.dataType
// This converts rows to the JSON output according to the given schema.
@transient
@@ -680,12 +674,12 @@ case class StructsToJson(
UTF8String.fromString(json)
}
- child.dataType match {
+ inputSchema match {
case _: StructType =>
(row: Any) =>
gen.write(row.asInstanceOf[InternalRow])
getAndReset()
- case ArrayType(_: StructType, _) =>
+ case _: ArrayType =>
(arr: Any) =>
gen.write(arr.asInstanceOf[ArrayData])
getAndReset()
@@ -693,34 +687,38 @@ case class StructsToJson(
(map: Any) =>
gen.write(map.asInstanceOf[MapData])
getAndReset()
- case ArrayType(_: MapType, _) =>
- (arr: Any) =>
- gen.write(arr.asInstanceOf[ArrayData])
- getAndReset()
}
}
override def dataType: DataType = StringType
- override def checkInputDataTypes(): TypeCheckResult = child.dataType match {
- case _: StructType | ArrayType(_: StructType, _) =>
+ override def checkInputDataTypes(): TypeCheckResult = inputSchema match {
+ case struct: StructType =>
try {
- JacksonUtils.verifySchema(rowSchema.asInstanceOf[StructType])
+ JacksonUtils.verifySchema(struct)
TypeCheckResult.TypeCheckSuccess
} catch {
case e: UnsupportedOperationException =>
TypeCheckResult.TypeCheckFailure(e.getMessage)
}
- case _: MapType | ArrayType(_: MapType, _) =>
+ case map: MapType =>
// TODO: let `JacksonUtils.verifySchema` verify a `MapType`
try {
- val st = StructType(StructField("a", rowSchema.asInstanceOf[MapType]) :: Nil)
+ val st = StructType(StructField("a", map) :: Nil)
JacksonUtils.verifySchema(st)
TypeCheckResult.TypeCheckSuccess
} catch {
case e: UnsupportedOperationException =>
TypeCheckResult.TypeCheckFailure(e.getMessage)
}
+ case array: ArrayType =>
+ try {
+ JacksonUtils.verifyType(prettyName, array)
+ TypeCheckResult.TypeCheckSuccess
+ } catch {
+ case e: UnsupportedOperationException =>
+ TypeCheckResult.TypeCheckFailure(e.getMessage)
+ }
case _ => TypeCheckResult.TypeCheckFailure(
s"Input type ${child.dataType.catalogString} must be a struct, array of structs or " +
"a map or array of map.")
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
index 7389477..9b86d86 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.catalyst.json
import java.io.Writer
-import java.nio.charset.StandardCharsets
import com.fasterxml.jackson.core._
@@ -28,7 +27,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, MapData}
import org.apache.spark.sql.types._
/**
- * `JackGenerator` can only be initialized with a `StructType` or a `MapType`.
+ * `JackGenerator` can only be initialized with a `StructType`, a `MapType` or an `ArrayType`.
* Once it is initialized with `StructType`, it can be used to write out a struct or an array of
* struct. Once it is initialized with `MapType`, it can be used to write out a map or an array
* of map. An exception will be thrown if trying to write out a struct if it is initialized with
@@ -43,34 +42,32 @@ private[sql] class JacksonGenerator(
// we can directly access data in `ArrayData` without the help of `SpecificMutableRow`.
private type ValueWriter = (SpecializedGetters, Int) => Unit
- // `JackGenerator` can only be initialized with a `StructType` or a `MapType`.
- require(dataType.isInstanceOf[StructType] || dataType.isInstanceOf[MapType],
- s"JacksonGenerator only supports to be initialized with a ${StructType.simpleString} " +
- s"or ${MapType.simpleString} but got ${dataType.catalogString}")
+ // `JackGenerator` can only be initialized with a `StructType`, a `MapType` or a `ArrayType`.
+ require(dataType.isInstanceOf[StructType] || dataType.isInstanceOf[MapType]
+ || dataType.isInstanceOf[ArrayType],
+ s"JacksonGenerator only supports to be initialized with a ${StructType.simpleString}, " +
+ s"${MapType.simpleString} or ${ArrayType.simpleString} but got ${dataType.catalogString}")
// `ValueWriter`s for all fields of the schema
private lazy val rootFieldWriters: Array[ValueWriter] = dataType match {
case st: StructType => st.map(_.dataType).map(makeWriter).toArray
case _ => throw new UnsupportedOperationException(
- s"Initial type ${dataType.catalogString} must be a struct")
+ s"Initial type ${dataType.catalogString} must be a ${StructType.simpleString}")
}
// `ValueWriter` for array data storing rows of the schema.
private lazy val arrElementWriter: ValueWriter = dataType match {
- case st: StructType =>
- (arr: SpecializedGetters, i: Int) => {
- writeObject(writeFields(arr.getStruct(i, st.length), st, rootFieldWriters))
- }
- case mt: MapType =>
- (arr: SpecializedGetters, i: Int) => {
- writeObject(writeMapData(arr.getMap(i), mt, mapElementWriter))
- }
+ case at: ArrayType => makeWriter(at.elementType)
+ case _: StructType | _: MapType => makeWriter(dataType)
+ case _ => throw new UnsupportedOperationException(
+ s"Initial type ${dataType.catalogString} must be " +
+ s"an ${ArrayType.simpleString}, a ${StructType.simpleString} or a ${MapType.simpleString}")
}
private lazy val mapElementWriter: ValueWriter = dataType match {
case mt: MapType => makeWriter(mt.valueType)
case _ => throw new UnsupportedOperationException(
- s"Initial type ${dataType.catalogString} must be a map")
+ s"Initial type ${dataType.catalogString} must be a ${MapType.simpleString}")
}
private val gen = new JsonFactory().createGenerator(writer).setRootValueSeparator(null)
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala
index f26b194..2d89c70 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala
@@ -32,11 +32,8 @@ object JacksonUtils {
}
}
- /**
- * Verify if the schema is supported in JSON parsing.
- */
- def verifySchema(schema: StructType): Unit = {
- def verifyType(name: String, dataType: DataType): Unit = dataType match {
+ def verifyType(name: String, dataType: DataType): Unit = {
+ dataType match {
case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType |
DoubleType | StringType | TimestampType | DateType | BinaryType | _: DecimalType =>
@@ -54,7 +51,12 @@ object JacksonUtils {
throw new UnsupportedOperationException(
s"Unable to convert column $name of type ${dataType.catalogString} to JSON.")
}
+ }
+ /**
+ * Verify if the schema is supported in JSON parsing.
+ */
+ def verifySchema(schema: StructType): Unit = {
schema.foreach(field => verifyType(field.name, field.dataType))
}
}
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index c120be4..10b67d7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -3612,11 +3612,11 @@ object functions {
def schema_of_json(e: Column): Column = withExpr(new SchemaOfJson(e.expr))
/**
- * (Scala-specific) Converts a column containing a `StructType`, `ArrayType` of `StructType`s,
- * a `MapType` or `ArrayType` of `MapType`s into a JSON string with the specified schema.
+ * (Scala-specific) Converts a column containing a `StructType`, `ArrayType` or
+ * a `MapType` into a JSON string with the specified schema.
* Throws an exception, in the case of an unsupported type.
*
- * @param e a column containing a struct or array of the structs.
+ * @param e a column containing a struct, an array or a map.
* @param options options to control how the struct column is converted into a json string.
* accepts the same options and the json data source.
*
@@ -3628,11 +3628,11 @@ object functions {
}
/**
- * (Java-specific) Converts a column containing a `StructType`, `ArrayType` of `StructType`s,
- * a `MapType` or `ArrayType` of `MapType`s into a JSON string with the specified schema.
+ * (Java-specific) Converts a column containing a `StructType`, `ArrayType` or
+ * a `MapType` into a JSON string with the specified schema.
* Throws an exception, in the case of an unsupported type.
*
- * @param e a column containing a struct or array of the structs.
+ * @param e a column containing a struct, an array or a map.
* @param options options to control how the struct column is converted into a json string.
* accepts the same options and the json data source.
*
@@ -3643,11 +3643,11 @@ object functions {
to_json(e, options.asScala.toMap)
/**
- * Converts a column containing a `StructType`, `ArrayType` of `StructType`s,
- * a `MapType` or `ArrayType` of `MapType`s into a JSON string with the specified schema.
+ * Converts a column containing a `StructType`, `ArrayType` or
+ * a `MapType` into a JSON string with the specified schema.
* Throws an exception, in the case of an unsupported type.
*
- * @param e a column containing a struct or array of the structs.
+ * @param e a column containing a struct, an array or a map.
*
* @group collection_funcs
* @since 2.1.0
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql
index 0cf370c..0f22c0e 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql
@@ -51,3 +51,8 @@ select from_json('[null, {"a":2}]', 'array<struct<a:int>>');
select from_json('[{"a": 1}, {"b":2}]', 'array<map<string,int>>');
select from_json('[{"a": 1}, 2]', 'array<map<string,int>>');
+
+-- to_json - array type
+select to_json(array('1', '2', '3'));
+select to_json(array(array(1, 2, 3), array(4)));
+
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
index 7444cdb..e550b43 100644
--- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 38
+-- Number of queries: 40
-- !query 0
@@ -9,7 +9,7 @@ struct<function_desc:string>
-- !query 0 output
Class: org.apache.spark.sql.catalyst.expressions.StructsToJson
Function: to_json
-Usage: to_json(expr[, options]) - Returns a json string with a given struct value
+Usage: to_json(expr[, options]) - Returns a JSON string with a given struct value
-- !query 1
@@ -38,7 +38,7 @@ Extended Usage:
Since: 2.2.0
Function: to_json
-Usage: to_json(expr[, options]) - Returns a json string with a given struct value
+Usage: to_json(expr[, options]) - Returns a JSON string with a given struct value
-- !query 2
@@ -354,3 +354,19 @@ select from_json('[{"a": 1}, 2]', 'array<map<string,int>>')
struct<jsontostructs([{"a": 1}, 2]):array<map<string,int>>>
-- !query 37 output
NULL
+
+
+-- !query 38
+select to_json(array('1', '2', '3'))
+-- !query 38 schema
+struct<structstojson(array(1, 2, 3)):string>
+-- !query 38 output
+["1","2","3"]
+
+
+-- !query 39
+select to_json(array(array(1, 2, 3), array(4)))
+-- !query 39 schema
+struct<structstojson(array(array(1, 2, 3), array(4))):string>
+-- !query 39 output
+[[1,2,3],[4]]
http://git-wip-us.apache.org/repos/asf/spark/blob/d749d034/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
index f321ab8..fe4bf15 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
@@ -469,4 +469,53 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext {
checkAnswer(sql("""select json[0] from jsonTable"""), Seq(Row(null)))
}
+
+ test("to_json - array of primitive types") {
+ val df = Seq(Array(1, 2, 3)).toDF("a")
+ checkAnswer(df.select(to_json($"a")), Seq(Row("[1,2,3]")))
+ }
+
+ test("roundtrip to_json -> from_json - array of primitive types") {
+ val arr = Array(1, 2, 3)
+ val df = Seq(arr).toDF("a")
+ checkAnswer(df.select(from_json(to_json($"a"), ArrayType(IntegerType))), Row(arr))
+ }
+
+ test("roundtrip from_json -> to_json - array of primitive types") {
+ val json = "[1,2,3]"
+ val df = Seq(json).toDF("a")
+ val schema = new ArrayType(IntegerType, false)
+
+ checkAnswer(df.select(to_json(from_json($"a", schema))), Seq(Row(json)))
+ }
+
+ test("roundtrip from_json -> to_json - array of arrays") {
+ val json = "[[1],[2,3],[4,5,6]]"
+ val jsonDF = Seq(json).toDF("a")
+ val schema = new ArrayType(ArrayType(IntegerType, false), false)
+
+ checkAnswer(
+ jsonDF.select(to_json(from_json($"a", schema))),
+ Seq(Row(json)))
+ }
+
+ test("roundtrip from_json -> to_json - array of maps") {
+ val json = """[{"a":1},{"b":2}]"""
+ val jsonDF = Seq(json).toDF("a")
+ val schema = new ArrayType(MapType(StringType, IntegerType, false), false)
+
+ checkAnswer(
+ jsonDF.select(to_json(from_json($"a", schema))),
+ Seq(Row(json)))
+ }
+
+ test("roundtrip from_json -> to_json - array of structs") {
+ val json = """[{"a":1},{"a":2},{"a":3}]"""
+ val jsonDF = Seq(json).toDF("a")
+ val schema = new ArrayType(new StructType().add("a", IntegerType), false)
+
+ checkAnswer(
+ jsonDF.select(to_json(from_json($"a", schema))),
+ Seq(Row(json)))
+ }
}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org