You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by ankits <an...@gmail.com> on 2015/07/21 21:41:29 UTC

Partition parquet data by ENUM column

Hi, I am using a custom build of spark 1.4 with the parquet dependency
upgraded to 1.7. I have thrift data encoded with parquet that i want to
partition by a column of type ENUM. Spark programming guide says partition
discovery is only supported for string and numeric columns, so it seems
partition discovery won't work out of the box here.

Is there any workaround that will allow me to partition by ENUMs? Will hive
partitioning help here? I am unfamiliar with Hive, and how it plays into
parquet, thrift and spark so I would appreciate any pointers in the right
direction. Thanks.



--
View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Partition-parquet-data-by-ENUM-column-tp23939.html
Sent from the Apache Spark User List mailing list archive at Nabble.com.

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


Re: Partition parquet data by ENUM column

Posted by Cheng Lian <li...@gmail.com>.
Your guess is partly right. Firstly, Spark SQL doesn’t have an 
equivalent data type to Parquet BINARY (ENUM), and always falls back to 
normal StringType. So in your case, Spark SQL just see a StringType, 
which maps to Parquet BINARY (UTF8), but the underlying data type is 
BINARY (ENUM).

Secondly, Parquet only supports filter push-down optimization for a 
limited set of data types 
<https://github.com/apache/parquet-mr/blob/apache-parquet-1.7.0/parquet-column/src/main/java/org/apache/parquet/filter2/predicate/ValidTypeMap.java#L66-L80>, 
in which ENUM is not included.

So the real problem here is that Spark SQL can’t prevent pushing down a 
predicate over an ENUM field since it sees the field as a normal string 
field. Would you mind to file a JIRA ticket?

Cheng

On 7/24/15 2:14 PM, Cheng Lian wrote:

> Could you please provide the full stack trace of the exception? And 
> what's the Git commit hash of the version you were using?
>
> Cheng
>
> On 7/24/15 6:37 AM, Jerry Lam wrote:
>> Hi Cheng,
>>
>> I ran into issues related to ENUM when I tried to use Filter push 
>> down. I'm using Spark 1.5.0 (which contains fixes for parquet filter 
>> push down). The exception is the following:
>>
>> java.lang.IllegalArgumentException: FilterPredicate column: item's 
>> declared type (org.apache.parquet.io.api.Binary) does not match the 
>> schema found in file metadata. Column item is of type: 
>> FullTypeDescriptor(PrimitiveType: BINARY, OriginalType: ENUM)
>> Valid types for this column are: null
>>
>> Is it because Spark does not recognize ENUM type in parquet?
>>
>> Best Regards,
>>
>> Jerry
>>
>>
>>
>> On Wed, Jul 22, 2015 at 12:21 AM, Cheng Lian <lian.cs.zju@gmail.com 
>> <ma...@gmail.com>> wrote:
>>
>>     On 7/22/15 9:03 AM, Ankit wrote:
>>
>>>     Thanks a lot Cheng. So it seems even in spark 1.3 and 1.4,
>>>     parquet ENUMs were treated as Strings in Spark SQL right? So
>>>     does this mean partitioning for enums already works in previous
>>>     versions too since they are just treated as strings?
>>
>>     It’s a little bit complicated. A Thrift/Avro/ProtoBuf |ENUM|
>>     value is represented as a |BINARY| annotated with original type
>>     |ENUM| in Parquet. For example, an optional |ENUM| field |e| is
>>     translated to something like |optional BINARY e (ENUM)| in
>>     Parquet. And the underlying data is always a UTF8 string of the
>>     |ENUM| name. However, the Parquet original type |ENUM| is not
>>     documented, thus Spark 1.3 and 1.4 doesn’t recognize the |ENUM|
>>     annotation and just see it as a normal |BINARY|. (I didn’t even
>>     notice the existence of |ENUM| in Parquet before PR #7048…)
>>
>>     On the other hand, Spark SQL has a boolean option named
>>     |spark.sql.parquet.binaryAsString|. When this option is set to
>>     |true|, all Parquet |BINARY| values are considered and converted
>>     to UTF8 strings. The original purpose of this option is used to
>>     work around a bug of Hive, which writes strings as plain Parquet
>>     |BINARY| values without a proper |UTF8| annotation.
>>
>>     That said, by using
>>     |sqlContext.setConf("spark.sql.parquet.binaryAsString", "true")|
>>     in Scala/Java/Python, or |SET
>>     spark.sql.parquet.binaryAsString=true| in SQL, you may read those
>>     |ENUM| values as plain UTF8 strings.
>>
>>>
>>>     Also, is there a good way to verify that the partitioning is
>>>     being used? I tried "explain" like (where data is partitioned by
>>>     "type" column)
>>>
>>>     scala> ev.filter("type = 'NON'").explain
>>>     == Physical Plan ==
>>>     Filter (type#4849 = NON)
>>>      PhysicalRDD [...cols..], MapPartitionsRDD[103] at map at
>>>     newParquet.scala:573
>>>
>>>     but that is the same even with non partitioned data.
>>
>>     Do you mean how to verify whether partition pruning is effective?
>>     You should be able to see log lines like this:
>>
>>         15/07/22 11:14:35 INFO DataSourceStrategy: Selected 1
>>         partitions out of 3, pruned 66.66666666666667% partitions.
>>
>>>
>>>
>>>     On Tue, Jul 21, 2015 at 4:35 PM, Cheng Lian
>>>     <lian.cs.zju@gmail.com <ma...@gmail.com>> wrote:
>>>
>>>         Parquet support for Thrift/Avro/ProtoBuf ENUM types are just
>>>         added to the master branch.
>>>         https://github.com/apache/spark/pull/7048
>>>
>>>         ENUM types are actually not in the Parquet format spec,
>>>         that's why we didn't have it at the first place. Basically,
>>>         ENUMs are always treated as UTF8 strings in Spark SQL now.
>>>
>>>         Cheng
>>>
>>>         On 7/22/15 3:41 AM, ankits wrote:
>>>
>>>             Hi, I am using a custom build of spark 1.4 with the
>>>             parquet dependency
>>>             upgraded to 1.7. I have thrift data encoded with parquet
>>>             that i want to
>>>             partition by a column of type ENUM. Spark programming
>>>             guide says partition
>>>             discovery is only supported for string and numeric
>>>             columns, so it seems
>>>             partition discovery won't work out of the box here.
>>>
>>>             Is there any workaround that will allow me to partition
>>>             by ENUMs? Will hive
>>>             partitioning help here? I am unfamiliar with Hive, and
>>>             how it plays into
>>>             parquet, thrift and spark so I would appreciate any
>>>             pointers in the right
>>>             direction. Thanks.
>>>
>>>
>>>
>>>             --
>>>             View this message in context:
>>>             http://apache-spark-user-list.1001560.n3.nabble.com/Partition-parquet-data-by-ENUM-column-tp23939.html
>>>             Sent from the Apache Spark User List mailing list
>>>             archive at Nabble.com.
>>>
>>>             ---------------------------------------------------------------------
>>>             To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
>>>             For additional commands, e-mail: user-help@spark.apache.org
>>>
>>>
>>>
>>>
>>     ​
>>
>>
>
​

​


Re: Partition parquet data by ENUM column

Posted by Cheng Lian <li...@gmail.com>.
Could you please provide the full stack trace of the exception? And 
what's the Git commit hash of the version you were using?

Cheng

On 7/24/15 6:37 AM, Jerry Lam wrote:
> Hi Cheng,
>
> I ran into issues related to ENUM when I tried to use Filter push 
> down. I'm using Spark 1.5.0 (which contains fixes for parquet filter 
> push down). The exception is the following:
>
> java.lang.IllegalArgumentException: FilterPredicate column: item's 
> declared type (org.apache.parquet.io.api.Binary) does not match the 
> schema found in file metadata. Column item is of type: 
> FullTypeDescriptor(PrimitiveType: BINARY, OriginalType: ENUM)
> Valid types for this column are: null
>
> Is it because Spark does not recognize ENUM type in parquet?
>
> Best Regards,
>
> Jerry
>
>
>
> On Wed, Jul 22, 2015 at 12:21 AM, Cheng Lian <lian.cs.zju@gmail.com 
> <ma...@gmail.com>> wrote:
>
>     On 7/22/15 9:03 AM, Ankit wrote:
>
>>     Thanks a lot Cheng. So it seems even in spark 1.3 and 1.4,
>>     parquet ENUMs were treated as Strings in Spark SQL right? So does
>>     this mean partitioning for enums already works in previous
>>     versions too since they are just treated as strings?
>
>     It’s a little bit complicated. A Thrift/Avro/ProtoBuf |ENUM| value
>     is represented as a |BINARY| annotated with original type |ENUM|
>     in Parquet. For example, an optional |ENUM| field |e| is
>     translated to something like |optional BINARY e (ENUM)| in
>     Parquet. And the underlying data is always a UTF8 string of the
>     |ENUM| name. However, the Parquet original type |ENUM| is not
>     documented, thus Spark 1.3 and 1.4 doesn’t recognize the |ENUM|
>     annotation and just see it as a normal |BINARY|. (I didn’t even
>     notice the existence of |ENUM| in Parquet before PR #7048…)
>
>     On the other hand, Spark SQL has a boolean option named
>     |spark.sql.parquet.binaryAsString|. When this option is set to
>     |true|, all Parquet |BINARY| values are considered and converted
>     to UTF8 strings. The original purpose of this option is used to
>     work around a bug of Hive, which writes strings as plain Parquet
>     |BINARY| values without a proper |UTF8| annotation.
>
>     That said, by using
>     |sqlContext.setConf("spark.sql.parquet.binaryAsString", "true")|
>     in Scala/Java/Python, or |SET
>     spark.sql.parquet.binaryAsString=true| in SQL, you may read those
>     |ENUM| values as plain UTF8 strings.
>
>>
>>     Also, is there a good way to verify that the partitioning is
>>     being used? I tried "explain" like (where data is partitioned by
>>     "type" column)
>>
>>     scala> ev.filter("type = 'NON'").explain
>>     == Physical Plan ==
>>     Filter (type#4849 = NON)
>>      PhysicalRDD [...cols..], MapPartitionsRDD[103] at map at
>>     newParquet.scala:573
>>
>>     but that is the same even with non partitioned data.
>
>     Do you mean how to verify whether partition pruning is effective?
>     You should be able to see log lines like this:
>
>         15/07/22 11:14:35 INFO DataSourceStrategy: Selected 1
>         partitions out of 3, pruned 66.66666666666667% partitions.
>
>>
>>
>>     On Tue, Jul 21, 2015 at 4:35 PM, Cheng Lian
>>     <lian.cs.zju@gmail.com <ma...@gmail.com>> wrote:
>>
>>         Parquet support for Thrift/Avro/ProtoBuf ENUM types are just
>>         added to the master branch.
>>         https://github.com/apache/spark/pull/7048
>>
>>         ENUM types are actually not in the Parquet format spec,
>>         that's why we didn't have it at the first place. Basically,
>>         ENUMs are always treated as UTF8 strings in Spark SQL now.
>>
>>         Cheng
>>
>>         On 7/22/15 3:41 AM, ankits wrote:
>>
>>             Hi, I am using a custom build of spark 1.4 with the
>>             parquet dependency
>>             upgraded to 1.7. I have thrift data encoded with parquet
>>             that i want to
>>             partition by a column of type ENUM. Spark programming
>>             guide says partition
>>             discovery is only supported for string and numeric
>>             columns, so it seems
>>             partition discovery won't work out of the box here.
>>
>>             Is there any workaround that will allow me to partition
>>             by ENUMs? Will hive
>>             partitioning help here? I am unfamiliar with Hive, and
>>             how it plays into
>>             parquet, thrift and spark so I would appreciate any
>>             pointers in the right
>>             direction. Thanks.
>>
>>
>>
>>             --
>>             View this message in context:
>>             http://apache-spark-user-list.1001560.n3.nabble.com/Partition-parquet-data-by-ENUM-column-tp23939.html
>>             Sent from the Apache Spark User List mailing list archive
>>             at Nabble.com.
>>
>>             ---------------------------------------------------------------------
>>             To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
>>             <ma...@spark.apache.org>
>>             For additional commands, e-mail:
>>             user-help@spark.apache.org
>>             <ma...@spark.apache.org>
>>
>>
>>
>>
>     ​
>
>


Re: Partition parquet data by ENUM column

Posted by Jerry Lam <ch...@gmail.com>.
Hi Cheng,

I ran into issues related to ENUM when I tried to use Filter push down. I'm
using Spark 1.5.0 (which contains fixes for parquet filter push down). The
exception is the following:

java.lang.IllegalArgumentException: FilterPredicate column: item's declared
type (org.apache.parquet.io.api.Binary) does not match the schema found in
file metadata. Column item is of type: FullTypeDescriptor(PrimitiveType:
BINARY, OriginalType: ENUM)
Valid types for this column are: null

Is it because Spark does not recognize ENUM type in parquet?

Best Regards,

Jerry



On Wed, Jul 22, 2015 at 12:21 AM, Cheng Lian <li...@gmail.com> wrote:

>  On 7/22/15 9:03 AM, Ankit wrote:
>
>   Thanks a lot Cheng. So it seems even in spark 1.3 and 1.4, parquet
> ENUMs were treated as Strings in Spark SQL right? So does this mean
> partitioning for enums already works in previous versions too since they
> are just treated as strings?
>
>   It’s a little bit complicated. A Thrift/Avro/ProtoBuf ENUM value is
> represented as a BINARY annotated with original type ENUM in Parquet. For
> example, an optional ENUM field e is translated to something like optional
> BINARY e (ENUM) in Parquet. And the underlying data is always a UTF8
> string of the ENUM name. However, the Parquet original type ENUM is not
> documented, thus Spark 1.3 and 1.4 doesn’t recognize the ENUM annotation
> and just see it as a normal BINARY. (I didn’t even notice the existence
> of ENUM in Parquet before PR #7048…)
>
> On the other hand, Spark SQL has a boolean option named
> spark.sql.parquet.binaryAsString. When this option is set to true, all
> Parquet BINARY values are considered and converted to UTF8 strings. The
> original purpose of this option is used to work around a bug of Hive, which
> writes strings as plain Parquet BINARY values without a proper UTF8
> annotation.
>
> That said, by using sqlContext.setConf("spark.sql.parquet.binaryAsString",
> "true") in Scala/Java/Python, or SET spark.sql.parquet.binaryAsString=true
> in SQL, you may read those ENUM values as plain UTF8 strings.
>
>
>  Also, is there a good way to verify that the partitioning is being used?
> I tried "explain" like (where data is partitioned by "type" column)
>
>  scala> ev.filter("type = 'NON'").explain
> == Physical Plan ==
> Filter (type#4849 = NON)
>  PhysicalRDD [...cols..], MapPartitionsRDD[103] at map at
> newParquet.scala:573
>
>  but that is the same even with non partitioned data.
>
>   Do you mean how to verify whether partition pruning is effective? You
> should be able to see log lines like this:
>
> 15/07/22 11:14:35 INFO DataSourceStrategy: Selected 1 partitions out of 3,
> pruned 66.66666666666667% partitions.
>
>
>
> On Tue, Jul 21, 2015 at 4:35 PM, Cheng Lian <li...@gmail.com> wrote:
>
>> Parquet support for Thrift/Avro/ProtoBuf ENUM types are just added to the
>> master branch. https://github.com/apache/spark/pull/7048
>>
>> ENUM types are actually not in the Parquet format spec, that's why we
>> didn't have it at the first place. Basically, ENUMs are always treated as
>> UTF8 strings in Spark SQL now.
>>
>> Cheng
>>
>> On 7/22/15 3:41 AM, ankits wrote:
>>
>>> Hi, I am using a custom build of spark 1.4 with the parquet dependency
>>> upgraded to 1.7. I have thrift data encoded with parquet that i want to
>>> partition by a column of type ENUM. Spark programming guide says
>>> partition
>>> discovery is only supported for string and numeric columns, so it seems
>>> partition discovery won't work out of the box here.
>>>
>>> Is there any workaround that will allow me to partition by ENUMs? Will
>>> hive
>>> partitioning help here? I am unfamiliar with Hive, and how it plays into
>>> parquet, thrift and spark so I would appreciate any pointers in the right
>>> direction. Thanks.
>>>
>>>
>>>
>>> --
>>>  View this message in context:
>>> <http://apache-spark-user-list.1001560.n3.nabble.com/Partition-parquet-data-by-ENUM-column-tp23939.html>
>>> http://apache-spark-user-list.1001560.n3.nabble.com/Partition-parquet-data-by-ENUM-column-tp23939.html
>>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>>
>>> ---------------------------------------------------------------------
>>>  To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
>>> For additional commands, e-mail: <us...@spark.apache.org>
>>> user-help@spark.apache.org
>>>
>>>
>>>
>>
>    ​
>

Re: Partition parquet data by ENUM column

Posted by Cheng Lian <li...@gmail.com>.
On 7/22/15 9:03 AM, Ankit wrote:

> Thanks a lot Cheng. So it seems even in spark 1.3 and 1.4, parquet 
> ENUMs were treated as Strings in Spark SQL right? So does this mean 
> partitioning for enums already works in previous versions too since 
> they are just treated as strings?

It’s a little bit complicated. A Thrift/Avro/ProtoBuf |ENUM| value is 
represented as a |BINARY| annotated with original type |ENUM| in 
Parquet. For example, an optional |ENUM| field |e| is translated to 
something like |optional BINARY e (ENUM)| in Parquet. And the underlying 
data is always a UTF8 string of the |ENUM| name. However, the Parquet 
original type |ENUM| is not documented, thus Spark 1.3 and 1.4 doesn’t 
recognize the |ENUM| annotation and just see it as a normal |BINARY|. (I 
didn’t even notice the existence of |ENUM| in Parquet before PR #7048…)

On the other hand, Spark SQL has a boolean option named 
|spark.sql.parquet.binaryAsString|. When this option is set to |true|, 
all Parquet |BINARY| values are considered and converted to UTF8 
strings. The original purpose of this option is used to work around a 
bug of Hive, which writes strings as plain Parquet |BINARY| values 
without a proper |UTF8| annotation.

That said, by using 
|sqlContext.setConf("spark.sql.parquet.binaryAsString", "true")| in 
Scala/Java/Python, or |SET spark.sql.parquet.binaryAsString=true| in 
SQL, you may read those |ENUM| values as plain UTF8 strings.

>
> Also, is there a good way to verify that the partitioning is being 
> used? I tried "explain" like (where data is partitioned by "type" column)
>
> scala> ev.filter("type = 'NON'").explain
> == Physical Plan ==
> Filter (type#4849 = NON)
>  PhysicalRDD [...cols..], MapPartitionsRDD[103] at map at 
> newParquet.scala:573
>
> but that is the same even with non partitioned data.

Do you mean how to verify whether partition pruning is effective? You 
should be able to see log lines like this:

    15/07/22 11:14:35 INFO DataSourceStrategy: Selected 1 partitions out
    of 3, pruned 66.66666666666667% partitions.

>
>
> On Tue, Jul 21, 2015 at 4:35 PM, Cheng Lian <lian.cs.zju@gmail.com 
> <ma...@gmail.com>> wrote:
>
>     Parquet support for Thrift/Avro/ProtoBuf ENUM types are just added
>     to the master branch. https://github.com/apache/spark/pull/7048
>
>     ENUM types are actually not in the Parquet format spec, that's why
>     we didn't have it at the first place. Basically, ENUMs are always
>     treated as UTF8 strings in Spark SQL now.
>
>     Cheng
>
>     On 7/22/15 3:41 AM, ankits wrote:
>
>         Hi, I am using a custom build of spark 1.4 with the parquet
>         dependency
>         upgraded to 1.7. I have thrift data encoded with parquet that
>         i want to
>         partition by a column of type ENUM. Spark programming guide
>         says partition
>         discovery is only supported for string and numeric columns, so
>         it seems
>         partition discovery won't work out of the box here.
>
>         Is there any workaround that will allow me to partition by
>         ENUMs? Will hive
>         partitioning help here? I am unfamiliar with Hive, and how it
>         plays into
>         parquet, thrift and spark so I would appreciate any pointers
>         in the right
>         direction. Thanks.
>
>
>
>         --
>         View this message in context:
>         http://apache-spark-user-list.1001560.n3.nabble.com/Partition-parquet-data-by-ENUM-column-tp23939.html
>         Sent from the Apache Spark User List mailing list archive at
>         Nabble.com.
>
>         ---------------------------------------------------------------------
>         To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
>         <ma...@spark.apache.org>
>         For additional commands, e-mail: user-help@spark.apache.org
>         <ma...@spark.apache.org>
>
>
>
>
​

Re: Partition parquet data by ENUM column

Posted by Ankit <an...@gmail.com>.
Thanks a lot Cheng. So it seems even in spark 1.3 and 1.4, parquet ENUMs
were treated as Strings in Spark SQL right? So does this mean partitioning
for enums already works in previous versions too since they are just
treated as strings?

Also, is there a good way to verify that the partitioning is being used? I
tried "explain" like (where data is partitioned by "type" column)

scala> ev.filter("type = 'NON'").explain
== Physical Plan ==
Filter (type#4849 = NON)
 PhysicalRDD [...cols..], MapPartitionsRDD[103] at map at
newParquet.scala:573

but that is the same even with non partitioned data.


On Tue, Jul 21, 2015 at 4:35 PM, Cheng Lian <li...@gmail.com> wrote:

> Parquet support for Thrift/Avro/ProtoBuf ENUM types are just added to the
> master branch. https://github.com/apache/spark/pull/7048
>
> ENUM types are actually not in the Parquet format spec, that's why we
> didn't have it at the first place. Basically, ENUMs are always treated as
> UTF8 strings in Spark SQL now.
>
> Cheng
>
> On 7/22/15 3:41 AM, ankits wrote:
>
>> Hi, I am using a custom build of spark 1.4 with the parquet dependency
>> upgraded to 1.7. I have thrift data encoded with parquet that i want to
>> partition by a column of type ENUM. Spark programming guide says partition
>> discovery is only supported for string and numeric columns, so it seems
>> partition discovery won't work out of the box here.
>>
>> Is there any workaround that will allow me to partition by ENUMs? Will
>> hive
>> partitioning help here? I am unfamiliar with Hive, and how it plays into
>> parquet, thrift and spark so I would appreciate any pointers in the right
>> direction. Thanks.
>>
>>
>>
>> --
>> View this message in context:
>> http://apache-spark-user-list.1001560.n3.nabble.com/Partition-parquet-data-by-ENUM-column-tp23939.html
>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
>> For additional commands, e-mail: user-help@spark.apache.org
>>
>>
>>
>

Re: Partition parquet data by ENUM column

Posted by Cheng Lian <li...@gmail.com>.
Parquet support for Thrift/Avro/ProtoBuf ENUM types are just added to 
the master branch. https://github.com/apache/spark/pull/7048

ENUM types are actually not in the Parquet format spec, that's why we 
didn't have it at the first place. Basically, ENUMs are always treated 
as UTF8 strings in Spark SQL now.

Cheng

On 7/22/15 3:41 AM, ankits wrote:
> Hi, I am using a custom build of spark 1.4 with the parquet dependency
> upgraded to 1.7. I have thrift data encoded with parquet that i want to
> partition by a column of type ENUM. Spark programming guide says partition
> discovery is only supported for string and numeric columns, so it seems
> partition discovery won't work out of the box here.
>
> Is there any workaround that will allow me to partition by ENUMs? Will hive
> partitioning help here? I am unfamiliar with Hive, and how it plays into
> parquet, thrift and spark so I would appreciate any pointers in the right
> direction. Thanks.
>
>
>
> --
> View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Partition-parquet-data-by-ENUM-column-tp23939.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: user-unsubscribe@spark.apache.org
> For additional commands, e-mail: user-help@spark.apache.org
>
>


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