Re: SparkSQL and multiple roots in 1.6

2016-03-25 Thread Spencer Uresk
Ok, that helped a lot - and I understand the feature/change better now.
Thank you!

On Fri, Mar 25, 2016 at 4:32 PM, Michael Armbrust 
wrote:

> Oh, I'm sorry I didn't fully understand what you were trying to do.  If
> you don't need partitioning, you can set
> "spark.sql.sources.partitionDiscovery.enabled=false".  Otherwise, I think
> you need to use the unioning approach.
>
> On Fri, Mar 25, 2016 at 1:35 PM, Spencer Uresk  wrote:
>
>> Thanks for the suggestion - I didn't try it at first because it seems
>> like I have multiple roots and not necessarily partitioned data. Is this
>> the correct way to do that?
>>
>> sqlContext.read.option("basePath",
>> "hdfs://user/hdfs/analytics/").json("hdfs://user/hdfs/analytics/*/PAGEVIEW/*/*")
>>
>> If so, it returns the same error:
>>
>> java.lang.AssertionError: assertion failed: Conflicting directory
>> structures detected. Suspicious paths:?
>> hdfs://user/hdfs/analytics/app1/PAGEVIEW
>> hdfs://user/hdfs/analytics/app2/PAGEVIEW
>>
>> On Fri, Mar 25, 2016 at 2:00 PM, Michael Armbrust > > wrote:
>>
>>> Have you tried setting a base path for partition discovery?
>>>
>>> Starting from Spark 1.6.0, partition discovery only finds partitions
 under the given paths by default. For the above example, if users pass
 path/to/table/gender=male to either SQLContext.read.parquet or
 SQLContext.read.load, gender will not be considered as a partitioning
 column. If users need to specify the base path that partition discovery
 should start with, they can set basePath in the data source options.
 For example, when path/to/table/gender=male is the path of the data
 and users set basePath to path/to/table/, gender will be a
 partitioning column.
>>>
>>>
>>>
>>> http://spark.apache.org/docs/latest/sql-programming-guide.html#partition-discovery
>>>
>>>
>>>
>>> On Fri, Mar 25, 2016 at 10:34 AM, Ted Yu  wrote:
>>>
 This is the original subject of the JIRA:
 Partition discovery fail if there is a _SUCCESS file in the table's
 root dir

 If I remember correctly, there were discussions on how (traditional)
 partition discovery slowed down Spark jobs.

 Cheers

 On Fri, Mar 25, 2016 at 10:15 AM, suresk  wrote:

> In previous versions of Spark, this would work:
>
> val events =
> sqlContext.jsonFile("hdfs://user/hdfs/analytics/*/PAGEVIEW/*/*")
>
> Where the first wildcard corresponds to an application directory, the
> second
> to a partition directory, and the third matched all the files in the
> partition directory. The records are all the exact same format, they
> are
> just broken out by application first, then event type. This
> functionality
> was really useful.
>
> In 1.6, this same call results in the following error:
>
> Conflicting directory structures detected. Suspicious paths:
> (list of paths)
>
> And then it recommends reading in each root directory separately and
> unioning them together. It looks like the change happened here:
>
> https://github.com/apache/spark/pull/9651
>
> 1) Simply out of curiosity, since I'm still fairly new to Spark - what
> is
> the benefit of no longer allowing multiple roots?
>
> 2) Is there a better way to do what I'm trying to do? Discovering all
> of the
> paths (I won't know them ahead of time), creating tables for each of
> them,
> and then doing all of the unions seems inefficient and a lot of extra
> work
> compared to what I had before.
>
> Thanks.
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/SparkSQL-and-multiple-roots-in-1-6-tp26598.html
> Sent from the Apache Spark User List mailing list archive at
> Nabble.com.
>
> -
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> For additional commands, e-mail: user-h...@spark.apache.org
>
>

>>>
>>
>


Re: SparkSQL and multiple roots in 1.6

2016-03-25 Thread Michael Armbrust
Oh, I'm sorry I didn't fully understand what you were trying to do.  If you
don't need partitioning, you can set
"spark.sql.sources.partitionDiscovery.enabled=false".  Otherwise, I think
you need to use the unioning approach.

On Fri, Mar 25, 2016 at 1:35 PM, Spencer Uresk  wrote:

> Thanks for the suggestion - I didn't try it at first because it seems like
> I have multiple roots and not necessarily partitioned data. Is this the
> correct way to do that?
>
> sqlContext.read.option("basePath",
> "hdfs://user/hdfs/analytics/").json("hdfs://user/hdfs/analytics/*/PAGEVIEW/*/*")
>
> If so, it returns the same error:
>
> java.lang.AssertionError: assertion failed: Conflicting directory
> structures detected. Suspicious paths:?
> hdfs://user/hdfs/analytics/app1/PAGEVIEW
> hdfs://user/hdfs/analytics/app2/PAGEVIEW
>
> On Fri, Mar 25, 2016 at 2:00 PM, Michael Armbrust 
> wrote:
>
>> Have you tried setting a base path for partition discovery?
>>
>> Starting from Spark 1.6.0, partition discovery only finds partitions
>>> under the given paths by default. For the above example, if users pass
>>> path/to/table/gender=male to either SQLContext.read.parquet or
>>> SQLContext.read.load, gender will not be considered as a partitioning
>>> column. If users need to specify the base path that partition discovery
>>> should start with, they can set basePath in the data source options.
>>> For example, when path/to/table/gender=male is the path of the data and
>>> users set basePath to path/to/table/, gender will be a partitioning
>>> column.
>>
>>
>>
>> http://spark.apache.org/docs/latest/sql-programming-guide.html#partition-discovery
>>
>>
>>
>> On Fri, Mar 25, 2016 at 10:34 AM, Ted Yu  wrote:
>>
>>> This is the original subject of the JIRA:
>>> Partition discovery fail if there is a _SUCCESS file in the table's root
>>> dir
>>>
>>> If I remember correctly, there were discussions on how (traditional)
>>> partition discovery slowed down Spark jobs.
>>>
>>> Cheers
>>>
>>> On Fri, Mar 25, 2016 at 10:15 AM, suresk  wrote:
>>>
 In previous versions of Spark, this would work:

 val events =
 sqlContext.jsonFile("hdfs://user/hdfs/analytics/*/PAGEVIEW/*/*")

 Where the first wildcard corresponds to an application directory, the
 second
 to a partition directory, and the third matched all the files in the
 partition directory. The records are all the exact same format, they are
 just broken out by application first, then event type. This
 functionality
 was really useful.

 In 1.6, this same call results in the following error:

 Conflicting directory structures detected. Suspicious paths:
 (list of paths)

 And then it recommends reading in each root directory separately and
 unioning them together. It looks like the change happened here:

 https://github.com/apache/spark/pull/9651

 1) Simply out of curiosity, since I'm still fairly new to Spark - what
 is
 the benefit of no longer allowing multiple roots?

 2) Is there a better way to do what I'm trying to do? Discovering all
 of the
 paths (I won't know them ahead of time), creating tables for each of
 them,
 and then doing all of the unions seems inefficient and a lot of extra
 work
 compared to what I had before.

 Thanks.



 --
 View this message in context:
 http://apache-spark-user-list.1001560.n3.nabble.com/SparkSQL-and-multiple-roots-in-1-6-tp26598.html
 Sent from the Apache Spark User List mailing list archive at Nabble.com.

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


>>>
>>
>


Re: SparkSQL and multiple roots in 1.6

2016-03-25 Thread Spencer Uresk
Thanks for the suggestion - I didn't try it at first because it seems like
I have multiple roots and not necessarily partitioned data. Is this the
correct way to do that?

sqlContext.read.option("basePath",
"hdfs://user/hdfs/analytics/").json("hdfs://user/hdfs/analytics/*/PAGEVIEW/*/*")

If so, it returns the same error:

java.lang.AssertionError: assertion failed: Conflicting directory
structures detected. Suspicious paths:?
hdfs://user/hdfs/analytics/app1/PAGEVIEW
hdfs://user/hdfs/analytics/app2/PAGEVIEW

On Fri, Mar 25, 2016 at 2:00 PM, Michael Armbrust 
wrote:

> Have you tried setting a base path for partition discovery?
>
> Starting from Spark 1.6.0, partition discovery only finds partitions under
>> the given paths by default. For the above example, if users pass
>> path/to/table/gender=male to either SQLContext.read.parquet or
>> SQLContext.read.load, gender will not be considered as a partitioning
>> column. If users need to specify the base path that partition discovery
>> should start with, they can set basePath in the data source options. For
>> example, when path/to/table/gender=male is the path of the data and
>> users set basePath to path/to/table/, gender will be a partitioning
>> column.
>
>
>
> http://spark.apache.org/docs/latest/sql-programming-guide.html#partition-discovery
>
>
>
> On Fri, Mar 25, 2016 at 10:34 AM, Ted Yu  wrote:
>
>> This is the original subject of the JIRA:
>> Partition discovery fail if there is a _SUCCESS file in the table's root
>> dir
>>
>> If I remember correctly, there were discussions on how (traditional)
>> partition discovery slowed down Spark jobs.
>>
>> Cheers
>>
>> On Fri, Mar 25, 2016 at 10:15 AM, suresk  wrote:
>>
>>> In previous versions of Spark, this would work:
>>>
>>> val events =
>>> sqlContext.jsonFile("hdfs://user/hdfs/analytics/*/PAGEVIEW/*/*")
>>>
>>> Where the first wildcard corresponds to an application directory, the
>>> second
>>> to a partition directory, and the third matched all the files in the
>>> partition directory. The records are all the exact same format, they are
>>> just broken out by application first, then event type. This functionality
>>> was really useful.
>>>
>>> In 1.6, this same call results in the following error:
>>>
>>> Conflicting directory structures detected. Suspicious paths:
>>> (list of paths)
>>>
>>> And then it recommends reading in each root directory separately and
>>> unioning them together. It looks like the change happened here:
>>>
>>> https://github.com/apache/spark/pull/9651
>>>
>>> 1) Simply out of curiosity, since I'm still fairly new to Spark - what is
>>> the benefit of no longer allowing multiple roots?
>>>
>>> 2) Is there a better way to do what I'm trying to do? Discovering all of
>>> the
>>> paths (I won't know them ahead of time), creating tables for each of
>>> them,
>>> and then doing all of the unions seems inefficient and a lot of extra
>>> work
>>> compared to what I had before.
>>>
>>> Thanks.
>>>
>>>
>>>
>>> --
>>> View this message in context:
>>> http://apache-spark-user-list.1001560.n3.nabble.com/SparkSQL-and-multiple-roots-in-1-6-tp26598.html
>>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>>
>>> -
>>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>>> For additional commands, e-mail: user-h...@spark.apache.org
>>>
>>>
>>
>


Re: SparkSQL and multiple roots in 1.6

2016-03-25 Thread Michael Armbrust
Have you tried setting a base path for partition discovery?

Starting from Spark 1.6.0, partition discovery only finds partitions under
> the given paths by default. For the above example, if users pass
> path/to/table/gender=male to either SQLContext.read.parquet or
> SQLContext.read.load, gender will not be considered as a partitioning
> column. If users need to specify the base path that partition discovery
> should start with, they can set basePath in the data source options. For
> example, when path/to/table/gender=male is the path of the data and users
> set basePath to path/to/table/, gender will be a partitioning column.


http://spark.apache.org/docs/latest/sql-programming-guide.html#partition-discovery



On Fri, Mar 25, 2016 at 10:34 AM, Ted Yu  wrote:

> This is the original subject of the JIRA:
> Partition discovery fail if there is a _SUCCESS file in the table's root
> dir
>
> If I remember correctly, there were discussions on how (traditional)
> partition discovery slowed down Spark jobs.
>
> Cheers
>
> On Fri, Mar 25, 2016 at 10:15 AM, suresk  wrote:
>
>> In previous versions of Spark, this would work:
>>
>> val events =
>> sqlContext.jsonFile("hdfs://user/hdfs/analytics/*/PAGEVIEW/*/*")
>>
>> Where the first wildcard corresponds to an application directory, the
>> second
>> to a partition directory, and the third matched all the files in the
>> partition directory. The records are all the exact same format, they are
>> just broken out by application first, then event type. This functionality
>> was really useful.
>>
>> In 1.6, this same call results in the following error:
>>
>> Conflicting directory structures detected. Suspicious paths:
>> (list of paths)
>>
>> And then it recommends reading in each root directory separately and
>> unioning them together. It looks like the change happened here:
>>
>> https://github.com/apache/spark/pull/9651
>>
>> 1) Simply out of curiosity, since I'm still fairly new to Spark - what is
>> the benefit of no longer allowing multiple roots?
>>
>> 2) Is there a better way to do what I'm trying to do? Discovering all of
>> the
>> paths (I won't know them ahead of time), creating tables for each of them,
>> and then doing all of the unions seems inefficient and a lot of extra work
>> compared to what I had before.
>>
>> Thanks.
>>
>>
>>
>> --
>> View this message in context:
>> http://apache-spark-user-list.1001560.n3.nabble.com/SparkSQL-and-multiple-roots-in-1-6-tp26598.html
>> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>>
>> -
>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> For additional commands, e-mail: user-h...@spark.apache.org
>>
>>
>


Re: SparkSQL and multiple roots in 1.6

2016-03-25 Thread Ted Yu
This is the original subject of the JIRA:
Partition discovery fail if there is a _SUCCESS file in the table's root dir

If I remember correctly, there were discussions on how (traditional)
partition discovery slowed down Spark jobs.

Cheers

On Fri, Mar 25, 2016 at 10:15 AM, suresk  wrote:

> In previous versions of Spark, this would work:
>
> val events =
> sqlContext.jsonFile("hdfs://user/hdfs/analytics/*/PAGEVIEW/*/*")
>
> Where the first wildcard corresponds to an application directory, the
> second
> to a partition directory, and the third matched all the files in the
> partition directory. The records are all the exact same format, they are
> just broken out by application first, then event type. This functionality
> was really useful.
>
> In 1.6, this same call results in the following error:
>
> Conflicting directory structures detected. Suspicious paths:
> (list of paths)
>
> And then it recommends reading in each root directory separately and
> unioning them together. It looks like the change happened here:
>
> https://github.com/apache/spark/pull/9651
>
> 1) Simply out of curiosity, since I'm still fairly new to Spark - what is
> the benefit of no longer allowing multiple roots?
>
> 2) Is there a better way to do what I'm trying to do? Discovering all of
> the
> paths (I won't know them ahead of time), creating tables for each of them,
> and then doing all of the unions seems inefficient and a lot of extra work
> compared to what I had before.
>
> Thanks.
>
>
>
> --
> View this message in context:
> http://apache-spark-user-list.1001560.n3.nabble.com/SparkSQL-and-multiple-roots-in-1-6-tp26598.html
> Sent from the Apache Spark User List mailing list archive at Nabble.com.
>
> -
> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
> For additional commands, e-mail: user-h...@spark.apache.org
>
>