[jira] [Commented] (SPARK-13219) Pushdown predicate propagation in SparkSQL with join

2016-02-23 Thread Evan Chan (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13219?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160340#comment-15160340
 ] 

Evan Chan commented on SPARK-13219:
---

[~smilegator] [~doodlegum] 

Guys, let me explain the strategy that we used to fix the join transitivity, 
which I believe is much more general and helps many more cases than the 
approach in PR 10490.

- First, we find out all of the join columns and all the tables that are joined 
for each join column.
- Next, we discover all the predicates (currently equals and IN, could be more) 
that filter on those join columns.
- We compute the joined tables for each join column which are missing the 
predicates
- We replicate the filter expression (using AND) for each missing table in the 
previous step.

The result is that no matter the number of tables and join columns, the 
predicates are augmented such that = and IN on literals are pushed to all the 
joined tables.

Only thing is the current code works on unanalyzed logical plans, so we need to 
port it to work on analyzed logical plans instead.

> Pushdown predicate propagation in SparkSQL with join
> 
>
> Key: SPARK-13219
> URL: https://issues.apache.org/jira/browse/SPARK-13219
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 1.4.1, 1.5.2, 1.6.0
> Environment: Spark 1.4
> Datastax Spark connector 1.4
> Cassandra. 2.1.12
> Centos 6.6
>Reporter: Abhinav Chawade
>
> When 2 or more tables are joined in SparkSQL and there is an equality clause 
> in query on attributes used to perform the join, it is useful to apply that 
> clause on scans for both table. If this is not done, one of the tables 
> results in full scan which can reduce the query dramatically. Consider 
> following example with 2 tables being joined.
> {code}
> CREATE TABLE assets (
> assetid int PRIMARY KEY,
> address text,
> propertyname text
> )
> CREATE TABLE tenants (
> assetid int PRIMARY KEY,
> name text
> )
> spark-sql> explain select t.name from tenants t, assets a where a.assetid = 
> t.assetid and t.assetid='1201';
> WARN  2016-02-05 23:05:19 org.apache.hadoop.util.NativeCodeLoader: Unable to 
> load native-hadoop library for your platform... using builtin-java classes 
> where applicable
> == Physical Plan ==
> Project [name#14]
>  ShuffledHashJoin [assetid#13], [assetid#15], BuildRight
>   Exchange (HashPartitioning 200)
>Filter (CAST(assetid#13, DoubleType) = 1201.0)
> HiveTableScan [assetid#13,name#14], (MetastoreRelation element, tenants, 
> Some(t)), None
>   Exchange (HashPartitioning 200)
>HiveTableScan [assetid#15], (MetastoreRelation element, assets, Some(a)), 
> None
> Time taken: 1.354 seconds, Fetched 8 row(s)
> {code}
> The simple workaround is to add another equality condition for each table but 
> it becomes cumbersome. It will be helpful if the query planner could improve 
> filter propagation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13219) Pushdown predicate propagation in SparkSQL with join

2016-02-23 Thread Evan Chan (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13219?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160333#comment-15160333
 ] 

Evan Chan commented on SPARK-13219:
---

Sorry, could you explain how SPARK-12957 affects this one?



> Pushdown predicate propagation in SparkSQL with join
> 
>
> Key: SPARK-13219
> URL: https://issues.apache.org/jira/browse/SPARK-13219
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 1.4.1, 1.5.2, 1.6.0
> Environment: Spark 1.4
> Datastax Spark connector 1.4
> Cassandra. 2.1.12
> Centos 6.6
>Reporter: Abhinav Chawade
>
> When 2 or more tables are joined in SparkSQL and there is an equality clause 
> in query on attributes used to perform the join, it is useful to apply that 
> clause on scans for both table. If this is not done, one of the tables 
> results in full scan which can reduce the query dramatically. Consider 
> following example with 2 tables being joined.
> {code}
> CREATE TABLE assets (
> assetid int PRIMARY KEY,
> address text,
> propertyname text
> )
> CREATE TABLE tenants (
> assetid int PRIMARY KEY,
> name text
> )
> spark-sql> explain select t.name from tenants t, assets a where a.assetid = 
> t.assetid and t.assetid='1201';
> WARN  2016-02-05 23:05:19 org.apache.hadoop.util.NativeCodeLoader: Unable to 
> load native-hadoop library for your platform... using builtin-java classes 
> where applicable
> == Physical Plan ==
> Project [name#14]
>  ShuffledHashJoin [assetid#13], [assetid#15], BuildRight
>   Exchange (HashPartitioning 200)
>Filter (CAST(assetid#13, DoubleType) = 1201.0)
> HiveTableScan [assetid#13,name#14], (MetastoreRelation element, tenants, 
> Some(t)), None
>   Exchange (HashPartitioning 200)
>HiveTableScan [assetid#15], (MetastoreRelation element, assets, Some(a)), 
> None
> Time taken: 1.354 seconds, Fetched 8 row(s)
> {code}
> The simple workaround is to add another equality condition for each table but 
> it becomes cumbersome. It will be helpful if the query planner could improve 
> filter propagation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13467) abstract python function to simplify pyspark code

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13467?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160330#comment-15160330
 ] 

Apache Spark commented on SPARK-13467:
--

User 'cloud-fan' has created a pull request for this issue:
https://github.com/apache/spark/pull/11342

> abstract python function to simplify pyspark code
> -
>
> Key: SPARK-13467
> URL: https://issues.apache.org/jira/browse/SPARK-13467
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Reporter: Wenchen Fan
>Priority: Trivial
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13467) abstract python function to simplify pyspark code

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13467?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13467:


Assignee: (was: Apache Spark)

> abstract python function to simplify pyspark code
> -
>
> Key: SPARK-13467
> URL: https://issues.apache.org/jira/browse/SPARK-13467
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Reporter: Wenchen Fan
>Priority: Trivial
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13467) abstract python function to simplify pyspark code

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13467?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13467:


Assignee: Apache Spark

> abstract python function to simplify pyspark code
> -
>
> Key: SPARK-13467
> URL: https://issues.apache.org/jira/browse/SPARK-13467
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Reporter: Wenchen Fan
>Assignee: Apache Spark
>Priority: Trivial
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (SPARK-13467) abstract python function to simplify pyspark code

2016-02-23 Thread Wenchen Fan (JIRA)
Wenchen Fan created SPARK-13467:
---

 Summary: abstract python function to simplify pyspark code
 Key: SPARK-13467
 URL: https://issues.apache.org/jira/browse/SPARK-13467
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Reporter: Wenchen Fan
Priority: Trivial






--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13466) Don't introduce redundant project with colum pruning rule

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13466:


Assignee: Apache Spark

> Don't introduce redundant project with colum pruning rule
> -
>
> Key: SPARK-13466
> URL: https://issues.apache.org/jira/browse/SPARK-13466
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Reporter: Liang-Chi Hsieh
>Assignee: Apache Spark
>
> With column pruning rule in optimizer, we will introduce redundant project 
> for some cases. We should prevent it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13466) Don't introduce redundant project with colum pruning rule

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13466?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160289#comment-15160289
 ] 

Apache Spark commented on SPARK-13466:
--

User 'viirya' has created a pull request for this issue:
https://github.com/apache/spark/pull/11341

> Don't introduce redundant project with colum pruning rule
> -
>
> Key: SPARK-13466
> URL: https://issues.apache.org/jira/browse/SPARK-13466
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Reporter: Liang-Chi Hsieh
>
> With column pruning rule in optimizer, we will introduce redundant project 
> for some cases. We should prevent it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13466) Don't introduce redundant project with colum pruning rule

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13466?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13466:


Assignee: (was: Apache Spark)

> Don't introduce redundant project with colum pruning rule
> -
>
> Key: SPARK-13466
> URL: https://issues.apache.org/jira/browse/SPARK-13466
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Reporter: Liang-Chi Hsieh
>
> With column pruning rule in optimizer, we will introduce redundant project 
> for some cases. We should prevent it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (SPARK-13466) Don't introduce redundant project with colum pruning rule

2016-02-23 Thread Liang-Chi Hsieh (JIRA)
Liang-Chi Hsieh created SPARK-13466:
---

 Summary: Don't introduce redundant project with colum pruning rule
 Key: SPARK-13466
 URL: https://issues.apache.org/jira/browse/SPARK-13466
 Project: Spark
  Issue Type: Bug
  Components: SQL
Reporter: Liang-Chi Hsieh


With column pruning rule in optimizer, we will introduce redundant project for 
some cases. We should prevent it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Comment Edited] (SPARK-12997) Use cast expression to perform type cast in csv

2016-02-23 Thread Hyukjin Kwon (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160270#comment-15160270
 ] 

Hyukjin Kwon edited comment on SPARK-12997 at 2/24/16 6:43 AM:
---

If I got this correctly, I think the issue itself is a little bit questionable 
at the end. If we can apply the same rules for casting, then it would be great 
to re-use the {{Cast()}} but both CSV and JSON data sources are dealing with 
data types differently.

As long as we are not going to change this behaviour for backwards 
compatibility, I believe this issue is invalid (or won't fix) or this will end 
up with some additional logics like I just did in the PR above with some cases 
to deal with the differences.


was (Author: hyukjin.kwon):
If I got this correctly, I think the issue itself is a little bit questionable 
at the end. If we can apply the same rules for casting, then it would be great 
to re-use the {{Cast()}} but both CSV and JSON data sources are dealing with 
data types differently.

As long as we are not going to change this behaviour for backwards 
compatibility, I believe this issue is invalid or this will end up with some 
additional logics like I just did in the PR above with some cases to deal with 
the differences.

> Use cast expression to perform type cast in csv
> ---
>
> Key: SPARK-12997
> URL: https://issues.apache.org/jira/browse/SPARK-12997
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Reporter: Reynold Xin
>
> CSVTypeCast.castTo should probably be removed, and just replace its usage 
> with a projection that uses a sequence of Cast expressions.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13450) SortMergeJoin will OOM when join rows have lot of same keys

2016-02-23 Thread Hong Shen (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13450?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160271#comment-15160271
 ] 

Hong Shen commented on SPARK-13450:
---

A join has a lot of rows with the same key.

> SortMergeJoin will OOM when join rows have lot of same keys
> ---
>
> Key: SPARK-13450
> URL: https://issues.apache.org/jira/browse/SPARK-13450
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.0
>Reporter: Hong Shen
>
>   When I run a sql with join, task throw  java.lang.OutOfMemoryError and sql 
> failed. I have set spark.executor.memory  4096m.
>   SortMergeJoin use a ArrayBuffer[InternalRow] to store bufferedMatches, if 
> the join rows have a lot of same key, it will throw OutOfMemoryError.
> {code}
>   /** Buffered rows from the buffered side of the join. This is empty if 
> there are no matches. */
>   private[this] val bufferedMatches: ArrayBuffer[InternalRow] = new 
> ArrayBuffer[InternalRow]
> {code}
>   Here is the stackTrace:
> {code}
> org.xerial.snappy.SnappyNative.arrayCopy(Native Method)
> org.xerial.snappy.Snappy.arrayCopy(Snappy.java:84)
> org.xerial.snappy.SnappyInputStream.rawRead(SnappyInputStream.java:190)
> org.xerial.snappy.SnappyInputStream.read(SnappyInputStream.java:163)
> java.io.DataInputStream.readFully(DataInputStream.java:195)
> java.io.DataInputStream.readLong(DataInputStream.java:416)
> org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillReader.loadNext(UnsafeSorterSpillReader.java:71)
> org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillMerger$2.loadNext(UnsafeSorterSpillMerger.java:79)
> org.apache.spark.sql.execution.UnsafeExternalRowSorter$1.next(UnsafeExternalRowSorter.java:136)
> org.apache.spark.sql.execution.UnsafeExternalRowSorter$1.next(UnsafeExternalRowSorter.java:123)
> org.apache.spark.sql.execution.RowIteratorFromScala.advanceNext(RowIterator.scala:84)
> org.apache.spark.sql.execution.joins.SortMergeJoinScanner.advancedBufferedToRowWithNullFreeJoinKey(SortMergeJoin.scala:300)
> org.apache.spark.sql.execution.joins.SortMergeJoinScanner.bufferMatchingRows(SortMergeJoin.scala:329)
> org.apache.spark.sql.execution.joins.SortMergeJoinScanner.findNextInnerJoinRows(SortMergeJoin.scala:229)
> org.apache.spark.sql.execution.joins.SortMergeJoin$$anonfun$doExecute$1$$anon$1.advanceNext(SortMergeJoin.scala:105)
> org.apache.spark.sql.execution.RowIteratorToScala.hasNext(RowIterator.scala:68)
> scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:88)
> org.apache.spark.sql.execution.aggregate.TungstenAggregate$$anonfun$doExecute$1$$anonfun$2.apply(TungstenAggregate.scala:86)
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:741)
> org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:741)
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:337)
> org.apache.spark.rdd.RDD.iterator(RDD.scala:301)
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
> org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:337)
> org.apache.spark.rdd.RDD.iterator(RDD.scala:301)
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
> org.apache.spark.scheduler.Task.run(Task.scala:89)
> org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:215)
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> java.lang.Thread.run(Thread.java:744)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-12997) Use cast expression to perform type cast in csv

2016-02-23 Thread Hyukjin Kwon (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12997?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160270#comment-15160270
 ] 

Hyukjin Kwon commented on SPARK-12997:
--

If I got this correctly, I think the issue itself is a little bit questionable 
at the end. If we can apply the same rules for casting, then it would be great 
to re-use the {{Cast()}} but both CSV and JSON data sources are dealing with 
data types differently.

As long as we are not going to change this behaviour for backwards 
compatibility, I believe this issue is invalid or this will end up with some 
additional logics like I just did in the PR above with some cases to deal with 
the differences.

> Use cast expression to perform type cast in csv
> ---
>
> Key: SPARK-12997
> URL: https://issues.apache.org/jira/browse/SPARK-12997
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Reporter: Reynold Xin
>
> CSVTypeCast.castTo should probably be removed, and just replace its usage 
> with a projection that uses a sequence of Cast expressions.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13307) TPCDS query 66 degraded by 30% in 1.6.0 compared to 1.4.1

2016-02-23 Thread Xiao Li (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13307?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160260#comment-15160260
 ] 

Xiao Li commented on SPARK-13307:
-

You need to check the plan and check the join type it is using. I guess the 
threshold 100 MB is still too low. 

BTW, 1.6 build also has a statistics-related issue. Recently, [~davies] just 
delivered a fix to resolve this problem.  

> TPCDS query 66 degraded by 30% in 1.6.0 compared to 1.4.1
> -
>
> Key: SPARK-13307
> URL: https://issues.apache.org/jira/browse/SPARK-13307
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.0
>Reporter: JESSE CHEN
>
> Majority of the TPCDS queries ran faster in 1.6.0 than in 1.4.1, average 
> about 9% faster. There are a few degraded, and one that is definitely not 
> within error margin is query 66.
> Query 66 in 1.4.1: 699 seconds
> Query 66 in 1.6.0: 918 seconds
> 30% worse.
> Collected the physical plans from both versions - drastic difference maybe 
> partially from using Tungsten in 1.6, but anything else at play here?
> Please see plans here:
> https://ibm.box.com/spark-sql-q66-debug-160plan
> https://ibm.box.com/spark-sql-q66-debug-141plan



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13307) TPCDS query 66 degraded by 30% in 1.6.0 compared to 1.4.1

2016-02-23 Thread Xiao Li (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13307?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160256#comment-15160256
 ] 

Xiao Li commented on SPARK-13307:
-

First, I am not sure if usage of broadcastjoin makes sense in this query, 
especially when your table size is huge. 

Second, are your queries written in SQL? or DataFrame APIs? Spark SQL does not 
provide broadcast hint for SQL users. If using DataFrame API, you can do 
something like 
{code}
df1.join(broadcast(df2), $"key" === $"key2", "leftsemi")
{code}

Third, I think performance regression is expected for avoiding OOM. 
SortMergeJoin is consuming less memory than ShuffleHashJoin. Thus, it might 
make more sense to choose SortMergeJoin as a default join type. 

> TPCDS query 66 degraded by 30% in 1.6.0 compared to 1.4.1
> -
>
> Key: SPARK-13307
> URL: https://issues.apache.org/jira/browse/SPARK-13307
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.0
>Reporter: JESSE CHEN
>
> Majority of the TPCDS queries ran faster in 1.6.0 than in 1.4.1, average 
> about 9% faster. There are a few degraded, and one that is definitely not 
> within error margin is query 66.
> Query 66 in 1.4.1: 699 seconds
> Query 66 in 1.6.0: 918 seconds
> 30% worse.
> Collected the physical plans from both versions - drastic difference maybe 
> partially from using Tungsten in 1.6, but anything else at play here?
> Please see plans here:
> https://ibm.box.com/spark-sql-q66-debug-160plan
> https://ibm.box.com/spark-sql-q66-debug-141plan



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13465) Add a task failure listener to TaskContext

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13465?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13465:


Assignee: Reynold Xin  (was: Apache Spark)

> Add a task failure listener to TaskContext
> --
>
> Key: SPARK-13465
> URL: https://issues.apache.org/jira/browse/SPARK-13465
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Reporter: Reynold Xin
>Assignee: Reynold Xin
>
> TaskContext supports task completion callback, which gets called regardless 
> of task failures. However, there is no way for the listener to know if there 
> is an error. This ticket proposes adding a new listener that gets called when 
> a task fails.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13465) Add a task failure listener to TaskContext

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160251#comment-15160251
 ] 

Apache Spark commented on SPARK-13465:
--

User 'rxin' has created a pull request for this issue:
https://github.com/apache/spark/pull/11340

> Add a task failure listener to TaskContext
> --
>
> Key: SPARK-13465
> URL: https://issues.apache.org/jira/browse/SPARK-13465
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Reporter: Reynold Xin
>Assignee: Reynold Xin
>
> TaskContext supports task completion callback, which gets called regardless 
> of task failures. However, there is no way for the listener to know if there 
> is an error. This ticket proposes adding a new listener that gets called when 
> a task fails.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-13465) Add a task failure listener to TaskContext

2016-02-23 Thread Reynold Xin (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13465?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Reynold Xin updated SPARK-13465:

Description: TaskContext supports task completion callback, which gets 
called regardless of task failures. However, there is no way for the listener 
to know if there is an error. This ticket proposes adding a new listener that 
gets called when a task fails.  (was: TaskContext supports task completion 
callback, which gets called regardless of task failures. However, there is no 
way for the listener to know if there is an error. This ticket proposes adding 
a new listener that adds a failure listener.
)

> Add a task failure listener to TaskContext
> --
>
> Key: SPARK-13465
> URL: https://issues.apache.org/jira/browse/SPARK-13465
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Reporter: Reynold Xin
>Assignee: Reynold Xin
>
> TaskContext supports task completion callback, which gets called regardless 
> of task failures. However, there is no way for the listener to know if there 
> is an error. This ticket proposes adding a new listener that gets called when 
> a task fails.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (SPARK-13465) Add a task failure listener to TaskContext

2016-02-23 Thread Reynold Xin (JIRA)
Reynold Xin created SPARK-13465:
---

 Summary: Add a task failure listener to TaskContext
 Key: SPARK-13465
 URL: https://issues.apache.org/jira/browse/SPARK-13465
 Project: Spark
  Issue Type: New Feature
  Components: SQL
Reporter: Reynold Xin
Assignee: Reynold Xin


TaskContext supports task completion callback, which gets called regardless of 
task failures. However, there is no way for the listener to know if there is an 
error. This ticket proposes adding a new listener that adds a failure listener.




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-13358) Retrieve grep path when doing Benchmark

2016-02-23 Thread Reynold Xin (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13358?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Reynold Xin updated SPARK-13358:

Assignee: Liang-Chi Hsieh

> Retrieve grep path when doing Benchmark
> ---
>
> Key: SPARK-13358
> URL: https://issues.apache.org/jira/browse/SPARK-13358
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Reporter: Liang-Chi Hsieh
>Assignee: Liang-Chi Hsieh
>Priority: Minor
> Fix For: 2.0.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Closed] (SPARK-12907) Use BitSet to represent null fields in ColumnVector

2016-02-23 Thread Kazuaki Ishizaki (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-12907?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kazuaki Ishizaki closed SPARK-12907.

Resolution: Won't Fix

> Use BitSet to represent null fields in ColumnVector
> ---
>
> Key: SPARK-12907
> URL: https://issues.apache.org/jira/browse/SPARK-12907
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 2.0.0
>Reporter: Kazuaki Ishizaki
>Priority: Minor
>
> Use bit vectors (BitSet) to represent null fields information in ColumnVector 
> to reduce memory footprint.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Resolved] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Davies Liu (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Davies Liu resolved SPARK-13431.

   Resolution: Fixed
Fix Version/s: 2.0.0

Issue resolved by pull request 11331
[https://github.com/apache/spark/pull/11331]

> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Priority: Blocker
> Fix For: 2.0.0
>
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13219) Pushdown predicate propagation in SparkSQL with join

2016-02-23 Thread Xiao Li (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13219?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160187#comment-15160187
 ] 

Xiao Li commented on SPARK-13219:
-

Thank you! Could you hold it now? I think we can do it in a better way after 
resolving https://issues.apache.org/jira/browse/SPARK-12957

> Pushdown predicate propagation in SparkSQL with join
> 
>
> Key: SPARK-13219
> URL: https://issues.apache.org/jira/browse/SPARK-13219
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Affects Versions: 1.4.1, 1.5.2, 1.6.0
> Environment: Spark 1.4
> Datastax Spark connector 1.4
> Cassandra. 2.1.12
> Centos 6.6
>Reporter: Abhinav Chawade
>
> When 2 or more tables are joined in SparkSQL and there is an equality clause 
> in query on attributes used to perform the join, it is useful to apply that 
> clause on scans for both table. If this is not done, one of the tables 
> results in full scan which can reduce the query dramatically. Consider 
> following example with 2 tables being joined.
> {code}
> CREATE TABLE assets (
> assetid int PRIMARY KEY,
> address text,
> propertyname text
> )
> CREATE TABLE tenants (
> assetid int PRIMARY KEY,
> name text
> )
> spark-sql> explain select t.name from tenants t, assets a where a.assetid = 
> t.assetid and t.assetid='1201';
> WARN  2016-02-05 23:05:19 org.apache.hadoop.util.NativeCodeLoader: Unable to 
> load native-hadoop library for your platform... using builtin-java classes 
> where applicable
> == Physical Plan ==
> Project [name#14]
>  ShuffledHashJoin [assetid#13], [assetid#15], BuildRight
>   Exchange (HashPartitioning 200)
>Filter (CAST(assetid#13, DoubleType) = 1201.0)
> HiveTableScan [assetid#13,name#14], (MetastoreRelation element, tenants, 
> Some(t)), None
>   Exchange (HashPartitioning 200)
>HiveTableScan [assetid#15], (MetastoreRelation element, assets, Some(a)), 
> None
> Time taken: 1.354 seconds, Fetched 8 row(s)
> {code}
> The simple workaround is to add another equality condition for each table but 
> it becomes cumbersome. It will be helpful if the query planner could improve 
> filter propagation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-12532) Join-key Pushdown via Predicate Transitivity

2016-02-23 Thread Xiao Li (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160186#comment-15160186
 ] 

Xiao Li commented on SPARK-12532:
-

Thank you! Could you hold it now? I think we can do it in a better way after 
resolving https://issues.apache.org/jira/browse/SPARK-12957

> Join-key Pushdown via Predicate Transitivity
> 
>
> Key: SPARK-12532
> URL: https://issues.apache.org/jira/browse/SPARK-12532
> Project: Spark
>  Issue Type: Improvement
>  Components: Optimizer, SQL
>Affects Versions: 1.6.0
>Reporter: Xiao Li
>  Labels: SQL
>
> {code}
> "SELECT * FROM upperCaseData JOIN lowerCaseData where lowerCaseData.n = 
> upperCaseData.N and lowerCaseData.n = 3"
> {code}
> {code}
> == Analyzed Logical Plan ==
> N: int, L: string, n: int, l: string
> Project [N#16,L#17,n#18,l#19]
> +- Filter ((n#18 = N#16) && (n#18 = 3))
>+- Join Inner, None
>   :- Subquery upperCaseData
>   :  +- LogicalRDD [N#16,L#17], MapPartitionsRDD[17] at beforeAll at 
> BeforeAndAfterAll.scala:187
>   +- Subquery lowerCaseData
>  +- LogicalRDD [n#18,l#19], MapPartitionsRDD[19] at beforeAll at 
> BeforeAndAfterAll.scala:187
> {code}
> Before the improvement, the optimized logical plan is
> {code}
> == Optimized Logical Plan ==
> Project [N#16,L#17,n#18,l#19]
> +- Join Inner, Some((n#18 = N#16))
>:- LogicalRDD [N#16,L#17], MapPartitionsRDD[17] at beforeAll at 
> BeforeAndAfterAll.scala:187
>+- Filter (n#18 = 3)
>   +- LogicalRDD [n#18,l#19], MapPartitionsRDD[19] at beforeAll at 
> BeforeAndAfterAll.scala:187
> {code}
> After the improvement, the optimized logical plan should be like
> {code}
> == Optimized Logical Plan ==
> Project [N#16,L#17,n#18,l#19]
> +- Join Inner, Some((n#18 = N#16))
>:- Filter (N#16 = 3)
>:  +- LogicalRDD [N#16,L#17], MapPartitionsRDD[17] at beforeAll at 
> BeforeAndAfterAll.scala:187
>+- Filter (n#18 = 3)
>   +- LogicalRDD [n#18,l#19], MapPartitionsRDD[19] at beforeAll at 
> BeforeAndAfterAll.scala:187
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13421) Make output of a SparkPlan configurable

2016-02-23 Thread Kazuaki Ishizaki (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13421?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160157#comment-15160157
 ] 

Kazuaki Ishizaki commented on SPARK-13421:
--

I am willing to work for and contribute to this.
Are there any design documents or prototype?

> Make output of a SparkPlan configurable
> ---
>
> Key: SPARK-13421
> URL: https://issues.apache.org/jira/browse/SPARK-13421
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Reporter: Herman van Hovell
>
> A SparkPlan currently outputs a iterator of {{InternalRow}}'s. This works 
> fine for many purposes. However an iterator is not a natural fit when we need 
> a (broadcasted) index or a {{ColumnarBatch}}.
> A SparkPlan should be able to define the shape/form/organization of the 
> output of its children.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13445) Seleting "data" with window function does not work unless aliased (using PARTITION BY)

2016-02-23 Thread Xiao Li (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13445?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160151#comment-15160151
 ] 

Xiao Li commented on SPARK-13445:
-

I will take a look at this. Thanks!

> Seleting "data" with window function does not work unless aliased (using 
> PARTITION BY)
> --
>
> Key: SPARK-13445
> URL: https://issues.apache.org/jira/browse/SPARK-13445
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Reporter: Reynold Xin
>Priority: Critical
>
> The code does not throw an exception if "data" is aliased.  Maybe this is a 
> reserved word or aliases are just required when using PARTITION BY?
> {code}
> sql("""
>   SELECT 
> data as the_data,
> row_number() over (partition BY data.type) AS foo
>   FROM event_record_sample
> """)
> {code}
> However, this code throws an error:
> {code}
> sql("""
>   SELECT 
> data,
> row_number() over (partition BY data.type) AS foo
>   FROM event_record_sample
> """)
> {code}
> {code}
> org.apache.spark.sql.AnalysisException: resolved attribute(s) type#15246 
> missing from 
> data#15107,par_cat#15112,schemaMajorVersion#15110,source#15108,recordId#15103,features#15106,eventType#15105,ts#15104L,schemaMinorVersion#15111,issues#15109
>  in operator !Project [data#15107,type#15246];
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:38)
>   at 
> org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:44)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:183)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:50)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:105)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at scala.collection.immutable.List.foreach(List.scala:318)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at scala.collection.immutable.List.foreach(List.scala:318)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at scala.collection.immutable.List.foreach(List.scala:318)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:50)
>   at 
> org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:44)
>   at 
> org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:34)
>   at org.apache.spark.sql.DataFrame.(DataFrame.scala:133)
>   at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52)
>   at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:816)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-1199) Type mismatch in Spark shell when using case class defined in shell

2016-02-23 Thread Prashant Sharma (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-1199?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160147#comment-15160147
 ] 

Prashant Sharma commented on SPARK-1199:


Did you try the :paste option ?

> Type mismatch in Spark shell when using case class defined in shell
> ---
>
> Key: SPARK-1199
> URL: https://issues.apache.org/jira/browse/SPARK-1199
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 0.9.0
>Reporter: Andrew Kerr
>Assignee: Prashant Sharma
>Priority: Blocker
> Fix For: 1.1.0
>
>
> *NOTE: This issue was fixed in 1.0.1, but the fix was reverted in Spark 1.0.2 
> pending further testing. The final fix will be in Spark 1.1.0.*
> Define a class in the shell:
> {code}
> case class TestClass(a:String)
> {code}
> and an RDD
> {code}
> val data = sc.parallelize(Seq("a")).map(TestClass(_))
> {code}
> define a function on it and map over the RDD
> {code}
> def itemFunc(a:TestClass):TestClass = a
> data.map(itemFunc)
> {code}
> Error:
> {code}
> :19: error: type mismatch;
>  found   : TestClass => TestClass
>  required: TestClass => ?
>   data.map(itemFunc)
> {code}
> Similarly with a mapPartitions:
> {code}
> def partitionFunc(a:Iterator[TestClass]):Iterator[TestClass] = a
> data.mapPartitions(partitionFunc)
> {code}
> {code}
> :19: error: type mismatch;
>  found   : Iterator[TestClass] => Iterator[TestClass]
>  required: Iterator[TestClass] => Iterator[?]
> Error occurred in an application involving default arguments.
>   data.mapPartitions(partitionFunc)
> {code}
> The behavior is the same whether in local mode or on a cluster.
> This isn't specific to RDDs. A Scala collection in the Spark shell has the 
> same problem.
> {code}
> scala> Seq(TestClass("foo")).map(itemFunc)
> :15: error: type mismatch;
>  found   : TestClass => TestClass
>  required: TestClass => ?
>   Seq(TestClass("foo")).map(itemFunc)
> ^
> {code}
> When run in the Scala console (not the Spark shell) there are no type 
> mismatch errors.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13358) Retrieve grep path when doing Benchmark

2016-02-23 Thread Liang-Chi Hsieh (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13358?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160145#comment-15160145
 ] 

Liang-Chi Hsieh commented on SPARK-13358:
-

[~davies] Can you help update the Assignee field? Thanks.

> Retrieve grep path when doing Benchmark
> ---
>
> Key: SPARK-13358
> URL: https://issues.apache.org/jira/browse/SPARK-13358
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Reporter: Liang-Chi Hsieh
>Priority: Minor
> Fix For: 2.0.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13464) Fix failed test test_reduce_by_key_and_window_with_none_invFunc in pyspark/streaming

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13464:


Assignee: Apache Spark

> Fix failed test test_reduce_by_key_and_window_with_none_invFunc in 
> pyspark/streaming
> 
>
> Key: SPARK-13464
> URL: https://issues.apache.org/jira/browse/SPARK-13464
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark, Streaming
>Reporter: Liang-Chi Hsieh
>Assignee: Apache Spark
>
> During backport a mllib feature, I found that the clearly checkouted 
> branch-1.3 codebase would fail at the test 
> {{test_reduce_by_key_and_window_with_none_invFunc}} in pyspark/streaming. We 
> should fix it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13464) Fix failed test test_reduce_by_key_and_window_with_none_invFunc in pyspark/streaming

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13464:


Assignee: (was: Apache Spark)

> Fix failed test test_reduce_by_key_and_window_with_none_invFunc in 
> pyspark/streaming
> 
>
> Key: SPARK-13464
> URL: https://issues.apache.org/jira/browse/SPARK-13464
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark, Streaming
>Reporter: Liang-Chi Hsieh
>
> During backport a mllib feature, I found that the clearly checkouted 
> branch-1.3 codebase would fail at the test 
> {{test_reduce_by_key_and_window_with_none_invFunc}} in pyspark/streaming. We 
> should fix it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13464) Fix failed test test_reduce_by_key_and_window_with_none_invFunc in pyspark/streaming

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160136#comment-15160136
 ] 

Apache Spark commented on SPARK-13464:
--

User 'viirya' has created a pull request for this issue:
https://github.com/apache/spark/pull/11339

> Fix failed test test_reduce_by_key_and_window_with_none_invFunc in 
> pyspark/streaming
> 
>
> Key: SPARK-13464
> URL: https://issues.apache.org/jira/browse/SPARK-13464
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark, Streaming
>Reporter: Liang-Chi Hsieh
>
> During backport a mllib feature, I found that the clearly checkouted 
> branch-1.3 codebase would fail at the test 
> {{test_reduce_by_key_and_window_with_none_invFunc}} in pyspark/streaming. We 
> should fix it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (SPARK-13464) Fix failed test test_reduce_by_key_and_window_with_none_invFunc in pyspark/streaming

2016-02-23 Thread Liang-Chi Hsieh (JIRA)
Liang-Chi Hsieh created SPARK-13464:
---

 Summary: Fix failed test 
test_reduce_by_key_and_window_with_none_invFunc in pyspark/streaming
 Key: SPARK-13464
 URL: https://issues.apache.org/jira/browse/SPARK-13464
 Project: Spark
  Issue Type: Bug
  Components: PySpark, Streaming
Reporter: Liang-Chi Hsieh


During backport a mllib feature, I found that the clearly checkouted branch-1.3 
codebase would fail at the test 
{{test_reduce_by_key_and_window_with_none_invFunc}} in pyspark/streaming. We 
should fix it.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-12878) Dataframe fails with nested User Defined Types

2016-02-23 Thread Jakob Odersky (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12878?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160119#comment-15160119
 ] 

Jakob Odersky commented on SPARK-12878:
---

I just tried your example and get a slightly different exception:

{{java.lang.ClassCastException: B cannot be cast to 
org.apache.spark.sql.catalyst.InternalRow}} (B as opposed to BoxedUnit)

However I actually don't understand why this worked in 1.5.2 in the first 
place. Consider the following extract from your snippet:
{code}
case A(list) =>
  val row = new GenericMutableRow(1)
  row.update(0, new GenericArrayData(list.map(_.asInstanceOf[Any]).toArray))
  row
{code}
although `list` is of type B in this case, I don't think that the B's are 
serialized according to the definition in BUDT.
I would assume you are solely responsible for the serialization and would have 
to call something like {{list.map(BUDT.serialize(_))}} to convert any child 
elements to an "SQL Datum" (not sure what that is but the docs say it, 
http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.sql.types.UserDefinedType)

Maybe someone with more knowledge on the topic can clarify what's going on?

> Dataframe fails with nested User Defined Types
> --
>
> Key: SPARK-12878
> URL: https://issues.apache.org/jira/browse/SPARK-12878
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.0
>Reporter: Joao
>Priority: Blocker
>
> Spark 1.6.0 crashes when using nested User Defined Types in a Dataframe. 
> In version 1.5.2 the code below worked just fine:
> import org.apache.spark.{SparkConf, SparkContext}
> import org.apache.spark.sql.catalyst.InternalRow
> import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
> import org.apache.spark.sql.types._
> @SQLUserDefinedType(udt = classOf[AUDT])
> case class A(list:Seq[B])
> class AUDT extends UserDefinedType[A] {
>   override def sqlType: DataType = StructType(Seq(StructField("list", 
> ArrayType(BUDT, containsNull = false), nullable = true)))
>   override def userClass: Class[A] = classOf[A]
>   override def serialize(obj: Any): Any = obj match {
> case A(list) =>
>   val row = new GenericMutableRow(1)
>   row.update(0, new 
> GenericArrayData(list.map(_.asInstanceOf[Any]).toArray))
>   row
>   }
>   override def deserialize(datum: Any): A = {
> datum match {
>   case row: InternalRow => new A(row.getArray(0).toArray(BUDT).toSeq)
> }
>   }
> }
> object AUDT extends AUDT
> @SQLUserDefinedType(udt = classOf[BUDT])
> case class B(text:Int)
> class BUDT extends UserDefinedType[B] {
>   override def sqlType: DataType = StructType(Seq(StructField("num", 
> IntegerType, nullable = false)))
>   override def userClass: Class[B] = classOf[B]
>   override def serialize(obj: Any): Any = obj match {
> case B(text) =>
>   val row = new GenericMutableRow(1)
>   row.setInt(0, text)
>   row
>   }
>   override def deserialize(datum: Any): B = {
> datum match {  case row: InternalRow => new B(row.getInt(0))  }
>   }
> }
> object BUDT extends BUDT
> object Test {
>   def main(args:Array[String]) = {
> val col = Seq(new A(Seq(new B(1), new B(2))),
>   new A(Seq(new B(3), new B(4
> val sc = new SparkContext(new 
> SparkConf().setMaster("local[1]").setAppName("TestSpark"))
> val sqlContext = new org.apache.spark.sql.SQLContext(sc)
> import sqlContext.implicits._
> val df = sc.parallelize(1 to 2 zip col).toDF("id","b")
> df.select("b").show()
> df.collect().foreach(println)
>   }
> }
> In the new version (1.6.0) I needed to include the following import:
> import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
> However, Spark crashes in runtime:
> 16/01/18 14:36:22 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
> java.lang.ClassCastException: scala.runtime.BoxedUnit cannot be cast to 
> org.apache.spark.sql.catalyst.InternalRow
>   at 
> org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getStruct(rows.scala:51)
>   at 
> org.apache.spark.sql.catalyst.expressions.GenericMutableRow.getStruct(rows.scala:248)
>   at 
> org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown
>  Source)
>   at 
> org.apache.spark.sql.execution.Project$$anonfun$1$$anonfun$apply$1.apply(basicOperators.scala:51)
>   at 
> org.apache.spark.sql.execution.Project$$anonfun$1$$anonfun$apply$1.apply(basicOperators.scala:49)
>   at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>   at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>   at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
>   at scala.collection.Iterator$$anon$10.next(Iterator.scala:312)
>   at 

[jira] [Updated] (SPARK-8369) Support dependency jar and files on HDFS in standalone cluster mode

2016-02-23 Thread Dong Lei (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-8369?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Dong Lei updated SPARK-8369:

Description: 
Currently, in standalone cluster mode, spark can take care of the app-jar 
whether the app-jar is specified by file:// or hdfs://. But the dependencies 
specified by --jars and --files do not support a hdfs:// prefix. 

For example:
spark-submit 
--master spark://ip:port
--deploy-mode cluster
 ...
--jars hdfs://path1/1.jar hdfs://path2/2.jar
--files hdfs://path3/3.file hdfs://path4/4.file
hdfs://path5/app.jar

only app.jar will be downloaded to the driver and distributed to executors, 
others (1.jar, 2.jar. 3.file, 4.file) will not. 
I think such a feature is useful for users. 


To support such a feature, I think we can treat the jars and files like the app 
jar in DriverRunner. We download them and replace the remote addresses with 
local addresses. And the DriverWrapper will not be aware.  

The problem is it's not easy to replace these addresses than replace the 
location app jar, because we have a placeholder for app jar "<>".  We 
may need to do some string matching to achieve it. 

  was:
Currently, in standalone cluster mode, spark can take care of the app-jar 
whether the app-jar is specified by file:// or hdfs://. But the dependencies 
specified by --jars and --files do not support a hdfs:// prefix. 

For example:
spark-submit 
 ...
--jars hdfs://path1/1.jar hdfs://path2/2.jar
--files hdfs://path3/3.file hdfs://path4/4.file
hdfs://path5/app.jar

only app.jar will be downloaded to the driver and distributed to executors, 
others (1.jar, 2.jar. 3.file, 4.file) will not. 
I think such a feature is useful for users. 


To support such a feature, I think we can treat the jars and files like the app 
jar in DriverRunner. We download them and replace the remote addresses with 
local addresses. And the DriverWrapper will not be aware.  

The problem is it's not easy to replace these addresses than replace the 
location app jar, because we have a placeholder for app jar "<>".  We 
may need to do some string matching to achieve it. 


> Support dependency jar and files on HDFS in standalone cluster mode
> ---
>
> Key: SPARK-8369
> URL: https://issues.apache.org/jira/browse/SPARK-8369
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Reporter: Dong Lei
>
> Currently, in standalone cluster mode, spark can take care of the app-jar 
> whether the app-jar is specified by file:// or hdfs://. But the dependencies 
> specified by --jars and --files do not support a hdfs:// prefix. 
> For example:
> spark-submit 
> --master spark://ip:port
> --deploy-mode cluster
>  ...
> --jars hdfs://path1/1.jar hdfs://path2/2.jar
> --files hdfs://path3/3.file hdfs://path4/4.file
> hdfs://path5/app.jar
> only app.jar will be downloaded to the driver and distributed to executors, 
> others (1.jar, 2.jar. 3.file, 4.file) will not. 
> I think such a feature is useful for users. 
> 
> To support such a feature, I think we can treat the jars and files like the 
> app jar in DriverRunner. We download them and replace the remote addresses 
> with local addresses. And the DriverWrapper will not be aware.  
> The problem is it's not easy to replace these addresses than replace the 
> location app jar, because we have a placeholder for app jar "<>".  
> We may need to do some string matching to achieve it. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-8369) Support dependency jar and files on HDFS in standalone cluster mode

2016-02-23 Thread Dong Lei (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-8369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15160097#comment-15160097
 ] 

Dong Lei commented on SPARK-8369:
-

Hi Jim, I was using standalone cluster mode. I will add them for clarity. 
Thanks for taking this issue.

> Support dependency jar and files on HDFS in standalone cluster mode
> ---
>
> Key: SPARK-8369
> URL: https://issues.apache.org/jira/browse/SPARK-8369
> Project: Spark
>  Issue Type: New Feature
>  Components: Spark Core
>Reporter: Dong Lei
>
> Currently, in standalone cluster mode, spark can take care of the app-jar 
> whether the app-jar is specified by file:// or hdfs://. But the dependencies 
> specified by --jars and --files do not support a hdfs:// prefix. 
> For example:
> spark-submit 
>  ...
> --jars hdfs://path1/1.jar hdfs://path2/2.jar
> --files hdfs://path3/3.file hdfs://path4/4.file
> hdfs://path5/app.jar
> only app.jar will be downloaded to the driver and distributed to executors, 
> others (1.jar, 2.jar. 3.file, 4.file) will not. 
> I think such a feature is useful for users. 
> 
> To support such a feature, I think we can treat the jars and files like the 
> app jar in DriverRunner. We download them and replace the remote addresses 
> with local addresses. And the DriverWrapper will not be aware.  
> The problem is it's not easy to replace these addresses than replace the 
> location app jar, because we have a placeholder for app jar "<>".  
> We may need to do some string matching to achieve it. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Resolved] (SPARK-13376) Improve column pruning

2016-02-23 Thread Davies Liu (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13376?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Davies Liu resolved SPARK-13376.

   Resolution: Fixed
Fix Version/s: 2.0.0

Issue resolved by pull request 11256
[https://github.com/apache/spark/pull/11256]

> Improve column pruning
> --
>
> Key: SPARK-13376
> URL: https://issues.apache.org/jira/browse/SPARK-13376
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Reporter: Davies Liu
>Assignee: Davies Liu
> Fix For: 2.0.0
>
>
> Column pruning could help to skip to columns that are not used by any 
> following operators.
> The current implementation only work with a few of logical plans, we should 
> improve that to support all of them. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13092) Track constraints in ExpressionSet

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13092?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13092:


Assignee: Michael Armbrust  (was: Apache Spark)

> Track constraints in ExpressionSet
> --
>
> Key: SPARK-13092
> URL: https://issues.apache.org/jira/browse/SPARK-13092
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Reporter: Sameer Agarwal
>Assignee: Michael Armbrust
>
> Create a new ExpressionSet that operates similar to an AttributeSet for 
> keeping track of constraints. A nice addition to this will be to try and have 
> it do other type of canonicalization (i.e. don't allow both a = b and b = a).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13092) Track constraints in ExpressionSet

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13092?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159988#comment-15159988
 ] 

Apache Spark commented on SPARK-13092:
--

User 'marmbrus' has created a pull request for this issue:
https://github.com/apache/spark/pull/11338

> Track constraints in ExpressionSet
> --
>
> Key: SPARK-13092
> URL: https://issues.apache.org/jira/browse/SPARK-13092
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Reporter: Sameer Agarwal
>Assignee: Michael Armbrust
>
> Create a new ExpressionSet that operates similar to an AttributeSet for 
> keeping track of constraints. A nice addition to this will be to try and have 
> it do other type of canonicalization (i.e. don't allow both a = b and b = a).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13092) Track constraints in ExpressionSet

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13092?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13092:


Assignee: Apache Spark  (was: Michael Armbrust)

> Track constraints in ExpressionSet
> --
>
> Key: SPARK-13092
> URL: https://issues.apache.org/jira/browse/SPARK-13092
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Reporter: Sameer Agarwal
>Assignee: Apache Spark
>
> Create a new ExpressionSet that operates similar to an AttributeSet for 
> keeping track of constraints. A nice addition to this will be to try and have 
> it do other type of canonicalization (i.e. don't allow both a = b and b = a).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13092) Track constraints in ExpressionSet

2016-02-23 Thread Michael Armbrust (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13092?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Michael Armbrust reassigned SPARK-13092:


Assignee: Michael Armbrust  (was: Sameer Agarwal)

> Track constraints in ExpressionSet
> --
>
> Key: SPARK-13092
> URL: https://issues.apache.org/jira/browse/SPARK-13092
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Reporter: Sameer Agarwal
>Assignee: Michael Armbrust
>
> Create a new ExpressionSet that operates similar to an AttributeSet for 
> keeping track of constraints. A nice addition to this will be to try and have 
> it do other type of canonicalization (i.e. don't allow both a = b and b = a).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (SPARK-13463) Support Column pruning for Dataset logical plan

2016-02-23 Thread Davies Liu (JIRA)
Davies Liu created SPARK-13463:
--

 Summary: Support Column pruning for Dataset logical plan
 Key: SPARK-13463
 URL: https://issues.apache.org/jira/browse/SPARK-13463
 Project: Spark
  Issue Type: Improvement
  Components: SQL
Reporter: Davies Liu


Column pruning may not work with some logical plan for Dataset, we need check 
that and make sure they works.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13441) NullPointerException when either HADOOP_CONF_DIR or YARN_CONF_DIR is not readable

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13441?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13441:


Assignee: Apache Spark

> NullPointerException when either HADOOP_CONF_DIR or YARN_CONF_DIR is not 
> readable
> -
>
> Key: SPARK-13441
> URL: https://issues.apache.org/jira/browse/SPARK-13441
> Project: Spark
>  Issue Type: Bug
>  Components: YARN
>Affects Versions: 1.4.1, 1.5.1, 1.6.0
>Reporter: Terence Yim
>Assignee: Apache Spark
>
> NPE is throw from the yarn Client.scala because {{File.listFiles()}} can 
> return null on directory that it doesn't have permission to list. This is the 
> code fragment in question:
> {noformat}
> // In org/apache/spark/deploy/yarn/Client.scala
> Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey =>
>   sys.env.get(envKey).foreach { path =>
> val dir = new File(path)
> if (dir.isDirectory()) {
>   // dir.listFiles() can return null
>   dir.listFiles().foreach { file =>
> if (file.isFile && !hadoopConfFiles.contains(file.getName())) {
>   hadoopConfFiles(file.getName()) = file
> }
>   }
> }
>   }
> }
> {noformat}
> To reproduce, simply do:
> {noformat}
> sudo mkdir /tmp/conf
> sudo chmod 700 /tmp/conf
> export HADOOP_CONF_DIR=/etc/hadoop/conf
> export YARN_CONF_DIR=/tmp/conf
> spark-submit --master yarn-client SimpleApp.py
> {noformat}
> It fails on any Spark app. Though not important, the SimpleApp.py I used 
> looks like this:
> {noformat}
> from pyspark import SparkContext
> sc = SparkContext(None, "Simple App")
> data = [1, 2, 3, 4, 5]
> distData = sc.parallelize(data)
> total = distData.reduce(lambda a, b: a + b)
> print("Total: %i" % total)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13441) NullPointerException when either HADOOP_CONF_DIR or YARN_CONF_DIR is not readable

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13441?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159954#comment-15159954
 ] 

Apache Spark commented on SPARK-13441:
--

User 'chtyim' has created a pull request for this issue:
https://github.com/apache/spark/pull/11337

> NullPointerException when either HADOOP_CONF_DIR or YARN_CONF_DIR is not 
> readable
> -
>
> Key: SPARK-13441
> URL: https://issues.apache.org/jira/browse/SPARK-13441
> Project: Spark
>  Issue Type: Bug
>  Components: YARN
>Affects Versions: 1.4.1, 1.5.1, 1.6.0
>Reporter: Terence Yim
>
> NPE is throw from the yarn Client.scala because {{File.listFiles()}} can 
> return null on directory that it doesn't have permission to list. This is the 
> code fragment in question:
> {noformat}
> // In org/apache/spark/deploy/yarn/Client.scala
> Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey =>
>   sys.env.get(envKey).foreach { path =>
> val dir = new File(path)
> if (dir.isDirectory()) {
>   // dir.listFiles() can return null
>   dir.listFiles().foreach { file =>
> if (file.isFile && !hadoopConfFiles.contains(file.getName())) {
>   hadoopConfFiles(file.getName()) = file
> }
>   }
> }
>   }
> }
> {noformat}
> To reproduce, simply do:
> {noformat}
> sudo mkdir /tmp/conf
> sudo chmod 700 /tmp/conf
> export HADOOP_CONF_DIR=/etc/hadoop/conf
> export YARN_CONF_DIR=/tmp/conf
> spark-submit --master yarn-client SimpleApp.py
> {noformat}
> It fails on any Spark app. Though not important, the SimpleApp.py I used 
> looks like this:
> {noformat}
> from pyspark import SparkContext
> sc = SparkContext(None, "Simple App")
> data = [1, 2, 3, 4, 5]
> distData = sc.parallelize(data)
> total = distData.reduce(lambda a, b: a + b)
> print("Total: %i" % total)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13441) NullPointerException when either HADOOP_CONF_DIR or YARN_CONF_DIR is not readable

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13441?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13441:


Assignee: (was: Apache Spark)

> NullPointerException when either HADOOP_CONF_DIR or YARN_CONF_DIR is not 
> readable
> -
>
> Key: SPARK-13441
> URL: https://issues.apache.org/jira/browse/SPARK-13441
> Project: Spark
>  Issue Type: Bug
>  Components: YARN
>Affects Versions: 1.4.1, 1.5.1, 1.6.0
>Reporter: Terence Yim
>
> NPE is throw from the yarn Client.scala because {{File.listFiles()}} can 
> return null on directory that it doesn't have permission to list. This is the 
> code fragment in question:
> {noformat}
> // In org/apache/spark/deploy/yarn/Client.scala
> Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR").foreach { envKey =>
>   sys.env.get(envKey).foreach { path =>
> val dir = new File(path)
> if (dir.isDirectory()) {
>   // dir.listFiles() can return null
>   dir.listFiles().foreach { file =>
> if (file.isFile && !hadoopConfFiles.contains(file.getName())) {
>   hadoopConfFiles(file.getName()) = file
> }
>   }
> }
>   }
> }
> {noformat}
> To reproduce, simply do:
> {noformat}
> sudo mkdir /tmp/conf
> sudo chmod 700 /tmp/conf
> export HADOOP_CONF_DIR=/etc/hadoop/conf
> export YARN_CONF_DIR=/tmp/conf
> spark-submit --master yarn-client SimpleApp.py
> {noformat}
> It fails on any Spark app. Though not important, the SimpleApp.py I used 
> looks like this:
> {noformat}
> from pyspark import SparkContext
> sc = SparkContext(None, "Simple App")
> data = [1, 2, 3, 4, 5]
> distData = sc.parallelize(data)
> total = distData.reduce(lambda a, b: a + b)
> print("Total: %i" % total)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Comment Edited] (SPARK-12422) Binding Spark Standalone Master to public IP fails

2016-02-23 Thread Jakob Odersky (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12422?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159904#comment-15159904
 ] 

Jakob Odersky edited comment on SPARK-12422 at 2/24/16 12:16 AM:
-

This blocker issue is quite old now, can you or anyone else still reproduce it?
I tried it in a non-docker environment (Debian 9) and everything worked fine 
(Spark versions 1.5.2 and 1.6.0).


was (Author: jodersky):
This blocker issue is quite old now, can you still reproduce it?
I tried it in a non-docker environment (Debian 9) and everything worked fine 
(Spark versions 1.5.2 and 1.6.0).

> Binding Spark Standalone Master to public IP fails
> --
>
> Key: SPARK-12422
> URL: https://issues.apache.org/jira/browse/SPARK-12422
> Project: Spark
>  Issue Type: Bug
>  Components: Deploy
>Affects Versions: 1.5.2
> Environment: Fails on direct deployment on Mac OSX and also in Docker 
> Environment (running on OSX or Ubuntu)
>Reporter: Bennet Jeutter
>Priority: Blocker
>
> The start of the Spark Standalone Master fails, when the host specified 
> equals the public IP address. For example I created a Docker Machine with 
> public IP 192.168.99.100, then I run:
> /usr/spark/bin/spark-class org.apache.spark.deploy.master.Master -h 
> 192.168.99.100
> It'll fail with:
> Exception in thread "main" java.net.BindException: Failed to bind to: 
> /192.168.99.100:7093: Service 'sparkMaster' failed after 16 retries!
>   at 
> org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:272)
>   at 
> akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:393)
>   at 
> akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:389)
>   at scala.util.Success$$anonfun$map$1.apply(Try.scala:206)
>   at scala.util.Try$.apply(Try.scala:161)
>   at scala.util.Success.map(Try.scala:206)
>   at scala.concurrent.Future$$anonfun$map$1.apply(Future.scala:235)
>   at scala.concurrent.Future$$anonfun$map$1.apply(Future.scala:235)
>   at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
>   at 
> akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
>   at 
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91)
>   at 
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
>   at 
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
>   at 
> scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
>   at 
> akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90)
>   at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>   at 
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
>   at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>   at 
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>   at 
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>   at 
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> So I thought oh well, lets just bind to the local IP and access it via public 
> IP - this doesn't work, it will give:
> dropping message [class akka.actor.ActorSelectionMessage] for non-local 
> recipient [Actor[akka.tcp://sparkMaster@192.168.99.100:7077/]] arriving at 
> [akka.tcp://sparkMaster@192.168.99.100:7077] inbound addresses are 
> [akka.tcp://sparkMaster@spark-master:7077]
> So there is currently no possibility to run all this... related stackoverflow 
> issues:
> * 
> http://stackoverflow.com/questions/31659228/getting-java-net-bindexception-when-attempting-to-start-spark-master-on-ec2-node
> * 
> http://stackoverflow.com/questions/33768029/access-apache-spark-standalone-master-via-ip



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-9325) Support `collect` on DataFrame columns

2016-02-23 Thread Oscar D. Lara Yejas (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-9325?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159928#comment-15159928
 ] 

Oscar D. Lara Yejas commented on SPARK-9325:


Hi, folks.

I have created a PR for this. A design document is enclosed in the PR.

Thanks,
Oscar

> Support `collect` on DataFrame columns
> --
>
> Key: SPARK-9325
> URL: https://issues.apache.org/jira/browse/SPARK-9325
> Project: Spark
>  Issue Type: Sub-task
>  Components: SparkR
>Reporter: Shivaram Venkataraman
>
> This is to support code of the form 
> ```
> ages <- collect(df$Age)
> ```
> Right now `df$Age` returns a Column, which has no functions supported.
> Similarly we might consider supporting `head(df$Age)` etc.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-9325) Support `collect` on DataFrame columns

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-9325?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-9325:
---

Assignee: Apache Spark

> Support `collect` on DataFrame columns
> --
>
> Key: SPARK-9325
> URL: https://issues.apache.org/jira/browse/SPARK-9325
> Project: Spark
>  Issue Type: Sub-task
>  Components: SparkR
>Reporter: Shivaram Venkataraman
>Assignee: Apache Spark
>
> This is to support code of the form 
> ```
> ages <- collect(df$Age)
> ```
> Right now `df$Age` returns a Column, which has no functions supported.
> Similarly we might consider supporting `head(df$Age)` etc.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-9325) Support `collect` on DataFrame columns

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-9325?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-9325:
---

Assignee: (was: Apache Spark)

> Support `collect` on DataFrame columns
> --
>
> Key: SPARK-9325
> URL: https://issues.apache.org/jira/browse/SPARK-9325
> Project: Spark
>  Issue Type: Sub-task
>  Components: SparkR
>Reporter: Shivaram Venkataraman
>
> This is to support code of the form 
> ```
> ages <- collect(df$Age)
> ```
> Right now `df$Age` returns a Column, which has no functions supported.
> Similarly we might consider supporting `head(df$Age)` etc.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-9325) Support `collect` on DataFrame columns

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-9325?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159917#comment-15159917
 ] 

Apache Spark commented on SPARK-9325:
-

User 'olarayej' has created a pull request for this issue:
https://github.com/apache/spark/pull/11336

> Support `collect` on DataFrame columns
> --
>
> Key: SPARK-9325
> URL: https://issues.apache.org/jira/browse/SPARK-9325
> Project: Spark
>  Issue Type: Sub-task
>  Components: SparkR
>Reporter: Shivaram Venkataraman
>
> This is to support code of the form 
> ```
> ages <- collect(df$Age)
> ```
> Right now `df$Age` returns a Column, which has no functions supported.
> Similarly we might consider supporting `head(df$Age)` etc.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-11381) Replace example code in mllib-linear-methods.md using include_example

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-11381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-11381:
--
Shepherd: Xusen Yin

> Replace example code in mllib-linear-methods.md using include_example
> -
>
> Key: SPARK-11381
> URL: https://issues.apache.org/jira/browse/SPARK-11381
> Project: Spark
>  Issue Type: Sub-task
>  Components: Documentation, MLlib
>Reporter: Xusen Yin
>Assignee: Dongjoon Hyun
>  Labels: starter
>
> This is similar to SPARK-11289 but for the example code in 
> mllib-linear-methods.md.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Resolved] (SPARK-10759) Missing Python code example in model selection user guide

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-10759?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng resolved SPARK-10759.
---
   Resolution: Fixed
Fix Version/s: 2.0.0

Issue resolved by pull request 11240
[https://github.com/apache/spark/pull/11240]

> Missing Python code example in model selection user guide
> -
>
> Key: SPARK-10759
> URL: https://issues.apache.org/jira/browse/SPARK-10759
> Project: Spark
>  Issue Type: Sub-task
>  Components: Documentation, ML
>Affects Versions: 1.5.0
>Reporter: Raela Wang
>Assignee: Jeremy
>Priority: Minor
>  Labels: starter
> Fix For: 2.0.0
>
>
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-cross-validation
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-train-validation-split



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-12422) Binding Spark Standalone Master to public IP fails

2016-02-23 Thread Jakob Odersky (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12422?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159904#comment-15159904
 ] 

Jakob Odersky commented on SPARK-12422:
---

This blocker issue is quite old now, can you still reproduce it?
I tried it in a non-docker environment (Debian 9) and everything worked fine 
(Spark versions 1.5.2 and 1.6.0).

> Binding Spark Standalone Master to public IP fails
> --
>
> Key: SPARK-12422
> URL: https://issues.apache.org/jira/browse/SPARK-12422
> Project: Spark
>  Issue Type: Bug
>  Components: Deploy
>Affects Versions: 1.5.2
> Environment: Fails on direct deployment on Mac OSX and also in Docker 
> Environment (running on OSX or Ubuntu)
>Reporter: Bennet Jeutter
>Priority: Blocker
>
> The start of the Spark Standalone Master fails, when the host specified 
> equals the public IP address. For example I created a Docker Machine with 
> public IP 192.168.99.100, then I run:
> /usr/spark/bin/spark-class org.apache.spark.deploy.master.Master -h 
> 192.168.99.100
> It'll fail with:
> Exception in thread "main" java.net.BindException: Failed to bind to: 
> /192.168.99.100:7093: Service 'sparkMaster' failed after 16 retries!
>   at 
> org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:272)
>   at 
> akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:393)
>   at 
> akka.remote.transport.netty.NettyTransport$$anonfun$listen$1.apply(NettyTransport.scala:389)
>   at scala.util.Success$$anonfun$map$1.apply(Try.scala:206)
>   at scala.util.Try$.apply(Try.scala:161)
>   at scala.util.Success.map(Try.scala:206)
>   at scala.concurrent.Future$$anonfun$map$1.apply(Future.scala:235)
>   at scala.concurrent.Future$$anonfun$map$1.apply(Future.scala:235)
>   at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
>   at 
> akka.dispatch.BatchingExecutor$AbstractBatch.processBatch(BatchingExecutor.scala:55)
>   at 
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply$mcV$sp(BatchingExecutor.scala:91)
>   at 
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
>   at 
> akka.dispatch.BatchingExecutor$BlockableBatch$$anonfun$run$1.apply(BatchingExecutor.scala:91)
>   at 
> scala.concurrent.BlockContext$.withBlockContext(BlockContext.scala:72)
>   at 
> akka.dispatch.BatchingExecutor$BlockableBatch.run(BatchingExecutor.scala:90)
>   at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:40)
>   at 
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
>   at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>   at 
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>   at 
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>   at 
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> So I thought oh well, lets just bind to the local IP and access it via public 
> IP - this doesn't work, it will give:
> dropping message [class akka.actor.ActorSelectionMessage] for non-local 
> recipient [Actor[akka.tcp://sparkMaster@192.168.99.100:7077/]] arriving at 
> [akka.tcp://sparkMaster@192.168.99.100:7077] inbound addresses are 
> [akka.tcp://sparkMaster@spark-master:7077]
> So there is currently no possibility to run all this... related stackoverflow 
> issues:
> * 
> http://stackoverflow.com/questions/31659228/getting-java-net-bindexception-when-attempting-to-start-spark-master-on-ec2-node
> * 
> http://stackoverflow.com/questions/33768029/access-apache-spark-standalone-master-via-ip



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-12877) TrainValidationSplit is missing in pyspark.ml.tuning

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12877?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159884#comment-15159884
 ] 

Apache Spark commented on SPARK-12877:
--

User 'JeremyNixon' has created a pull request for this issue:
https://github.com/apache/spark/pull/11335

> TrainValidationSplit is missing in pyspark.ml.tuning
> 
>
> Key: SPARK-12877
> URL: https://issues.apache.org/jira/browse/SPARK-12877
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark
>Affects Versions: 1.6.0
>Reporter: Wojciech Jurczyk
>
> I was investingating progress in SPARK-10759 and I noticed that there is no 
> TrainValidationSplit class in pyspark.ml.tuning module.
> Java/Scala's examples SPARK-10759 use 
> org.apache.spark.ml.tuning.TrainValidationSplit that is not available from 
> Python and this blocks SPARK-10759.
> Does the class have different name in PySpark, maybe? Also, I couldn't find 
> any JIRA task to saying it need to be implemented. Is it by design that the 
> TrainValidationSplit estimator is not ported to PySpark? If not, that is if 
> the estimator needs porting then I would like to contribute.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-12877) TrainValidationSplit is missing in pyspark.ml.tuning

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-12877?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-12877:


Assignee: (was: Apache Spark)

> TrainValidationSplit is missing in pyspark.ml.tuning
> 
>
> Key: SPARK-12877
> URL: https://issues.apache.org/jira/browse/SPARK-12877
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark
>Affects Versions: 1.6.0
>Reporter: Wojciech Jurczyk
>
> I was investingating progress in SPARK-10759 and I noticed that there is no 
> TrainValidationSplit class in pyspark.ml.tuning module.
> Java/Scala's examples SPARK-10759 use 
> org.apache.spark.ml.tuning.TrainValidationSplit that is not available from 
> Python and this blocks SPARK-10759.
> Does the class have different name in PySpark, maybe? Also, I couldn't find 
> any JIRA task to saying it need to be implemented. Is it by design that the 
> TrainValidationSplit estimator is not ported to PySpark? If not, that is if 
> the estimator needs porting then I would like to contribute.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Resolved] (SPARK-13011) K-means wrapper in SparkR

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13011?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng resolved SPARK-13011.
---
   Resolution: Fixed
Fix Version/s: 2.0.0

Issue resolved by pull request 11124
[https://github.com/apache/spark/pull/11124]

> K-means wrapper in SparkR
> -
>
> Key: SPARK-13011
> URL: https://issues.apache.org/jira/browse/SPARK-13011
> Project: Spark
>  Issue Type: New Feature
>  Components: ML, SparkR
>Reporter: Xiangrui Meng
>Assignee: Xusen Yin
> Fix For: 2.0.0
>
>
> Implement a simple wrapper in SparkR to support k-means.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Stavros Kontopoulos (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159862#comment-15159862
 ] 

Stavros Kontopoulos commented on SPARK-13431:
-

cool!

> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Priority: Blocker
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-13410) unionAll AnalysisException with DataFrames containing UDT columns.

2016-02-23 Thread Reynold Xin (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13410?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Reynold Xin updated SPARK-13410:

Fix Version/s: 1.6.1

> unionAll AnalysisException with DataFrames containing UDT columns.
> --
>
> Key: SPARK-13410
> URL: https://issues.apache.org/jira/browse/SPARK-13410
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.5.0, 1.6.0
>Reporter: Franklyn Dsouza
>Assignee: Franklyn Dsouza
>  Labels: patch
> Fix For: 1.6.1, 2.0.0
>
>   Original Estimate: 3h
>  Remaining Estimate: 3h
>
> Unioning two DataFrames that contain UDTs fails with 
> {quote}
> AnalysisException: u"unresolved operator 'Union;"
> {quote}
> I tracked this down to this line 
> https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala#L202
> Which compares datatypes between the output attributes of both logical plans. 
> However for UDTs this will be a new instance of the UserDefinedType or 
> PythonUserDefinedType 
> https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala#L158
>  
> So this equality check will check if the two instances are the same and since 
> they aren't references to a singleton this check fails. 
> *Note: this will work fine if you are unioning the dataframe with itself.*
> I have a proposed patch for this which overrides the equality operator on the 
> two classes here: https://github.com/apache/spark/pull/11279
> Reproduction steps
> {code}
> from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
> from pyspark.sql import types
> schema = types.StructType([types.StructField("point", PythonOnlyUDT(), True)])
> #note they need to be two separate dataframes
> a = sqlCtx.createDataFrame([[PythonOnlyPoint(1.0, 2.0)]], schema)
> b = sqlCtx.createDataFrame([[PythonOnlyPoint(3.0, 4.0)]], schema)
> c = a.unionAll(b)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-13461) Duplicated example code merge and cleanup

2016-02-23 Thread Xusen Yin (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xusen Yin updated SPARK-13461:
--
Description: 
Merge duplicated code after we finishing the example code substitution.

Duplications include:

* JavaTrainValidationSplitExample 

* TrainValidationSplitExample

* Others can be added here ...

  was:
Merge duplicated code after we finishing the example code substitution.

Duplications include:

* JavaTrainValidationSplitExample 

* TrainValidationSplitExample


> Duplicated example code merge and cleanup
> -
>
> Key: SPARK-13461
> URL: https://issues.apache.org/jira/browse/SPARK-13461
> Project: Spark
>  Issue Type: Sub-task
>  Components: Documentation
>Reporter: Xusen Yin
>Priority: Minor
>  Labels: starter
>
> Merge duplicated code after we finishing the example code substitution.
> Duplications include:
> * JavaTrainValidationSplitExample 
> * TrainValidationSplitExample
> * Others can be added here ...



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-13461) Duplicated example code merge and cleanup

2016-02-23 Thread Xusen Yin (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13461?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xusen Yin updated SPARK-13461:
--
Description: 
Merge duplicated code after we finishing the example code substitution.

Duplications include:

* JavaTrainValidationSplitExample 

* TrainValidationSplitExample

  was:Merge duplicated code after we finishing the example code substitution.


> Duplicated example code merge and cleanup
> -
>
> Key: SPARK-13461
> URL: https://issues.apache.org/jira/browse/SPARK-13461
> Project: Spark
>  Issue Type: Sub-task
>  Components: Documentation
>Reporter: Xusen Yin
>Priority: Minor
>  Labels: starter
>
> Merge duplicated code after we finishing the example code substitution.
> Duplications include:
> * JavaTrainValidationSplitExample 
> * TrainValidationSplitExample



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13390) Java Spark createDataFrame with List parameter bug

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13390?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159832#comment-15159832
 ] 

Apache Spark commented on SPARK-13390:
--

User 'zsxwing' has created a pull request for this issue:
https://github.com/apache/spark/pull/11334

> Java Spark createDataFrame with List parameter bug
> --
>
> Key: SPARK-13390
> URL: https://issues.apache.org/jira/browse/SPARK-13390
> Project: Spark
>  Issue Type: Bug
>  Components: Spark Core
>Affects Versions: 1.6.0
> Environment: Java spark, Linux
>Reporter: mike niemaz
>Assignee: Shixiong Zhu
>
> I noticed the following bug while testing the dataframe SQL join capabilities.
> Instructions to reproduce it:
> - Read a text file from local file system using JavaSparkContext#texFile 
> method
> - Create a list of related custom objects based on the previously created 
> JavaRDD, using the map function
> -  Create a dataframe using SQLContext createDataFrame(java.util.List, Class) 
> method
>  - Count the dataframe elements using dataframe#count method
> It crashes with the following stacktrace error:
> org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
> TungstenAggregate(key=[], functions=[(count(1),mode=Final,isDistinct=false)], 
> output=[count#7L])
> +- TungstenExchange SinglePartition, None
>+- TungstenAggregate(key=[], 
> functions=[(count(1),mode=Partial,isDistinct=false)], output=[count#10L])
>   +- LocalTableScan [[empty row],[empty row],[empty row],[empty 
> row],[empty row],[empty row],[empty row],[empty row],[empty row],[empty row]]
>   at 
> org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49)
>   at 
> org.apache.spark.sql.execution.aggregate.TungstenAggregate.doExecute(TungstenAggregate.scala:80)
>   at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132)
>   at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130)
>   at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
>   at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130)
>   at 
> org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:166)
>   at 
> org.apache.spark.sql.execution.SparkPlan.executeCollectPublic(SparkPlan.scala:174)
>   at 
> org.apache.spark.sql.DataFrame$$anonfun$org$apache$spark$sql$DataFrame$$execute$1$1.apply(DataFrame.scala:1538)
>   at 
> org.apache.spark.sql.DataFrame$$anonfun$org$apache$spark$sql$DataFrame$$execute$1$1.apply(DataFrame.scala:1538)
>   at 
> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:56)
>   at 
> org.apache.spark.sql.DataFrame.withNewExecutionId(DataFrame.scala:2125)
>   at 
> org.apache.spark.sql.DataFrame.org$apache$spark$sql$DataFrame$$execute$1(DataFrame.scala:1537)
>   at 
> org.apache.spark.sql.DataFrame.org$apache$spark$sql$DataFrame$$collect(DataFrame.scala:1544)
>   at 
> org.apache.spark.sql.DataFrame$$anonfun$count$1.apply(DataFrame.scala:1554)
>   at 
> org.apache.spark.sql.DataFrame$$anonfun$count$1.apply(DataFrame.scala:1553)
>   at org.apache.spark.sql.DataFrame.withCallback(DataFrame.scala:2138)
>   at org.apache.spark.sql.DataFrame.count(DataFrame.scala:1553)
>   at injection.EMATests.joinTest1(EMATests.java:259)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:498)
>   at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
>   at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>   at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
>   at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>   at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
>   at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
>   at 
> org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
>   at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
>   at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
>   at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
>   at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
>   at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
>   at 
> 

[jira] [Resolved] (SPARK-13373) Generate code for sort merge join

2016-02-23 Thread Davies Liu (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13373?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Davies Liu resolved SPARK-13373.

   Resolution: Fixed
Fix Version/s: 2.0.0

Issue resolved by pull request 11248
[https://github.com/apache/spark/pull/11248]

> Generate code for sort merge join
> -
>
> Key: SPARK-13373
> URL: https://issues.apache.org/jira/browse/SPARK-13373
> Project: Spark
>  Issue Type: New Feature
>  Components: SQL
>Reporter: Davies Liu
>Assignee: Davies Liu
> Fix For: 2.0.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-12877) TrainValidationSplit is missing in pyspark.ml.tuning

2016-02-23 Thread Xusen Yin (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12877?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159822#comment-15159822
 ] 

Xusen Yin commented on SPARK-12877:
---

Yes, we lack TrainValidationSplit in Python. Go ahead porting it.

FYI [~mengxr] [~josephkb] and [~yanboliang].

> TrainValidationSplit is missing in pyspark.ml.tuning
> 
>
> Key: SPARK-12877
> URL: https://issues.apache.org/jira/browse/SPARK-12877
> Project: Spark
>  Issue Type: Bug
>  Components: PySpark
>Affects Versions: 1.6.0
>Reporter: Wojciech Jurczyk
>
> I was investingating progress in SPARK-10759 and I noticed that there is no 
> TrainValidationSplit class in pyspark.ml.tuning module.
> Java/Scala's examples SPARK-10759 use 
> org.apache.spark.ml.tuning.TrainValidationSplit that is not available from 
> Python and this blocks SPARK-10759.
> Does the class have different name in PySpark, maybe? Also, I couldn't find 
> any JIRA task to saying it need to be implemented. Is it by design that the 
> TrainValidationSplit estimator is not ported to PySpark? If not, that is if 
> the estimator needs porting then I would like to contribute.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (SPARK-13462) Vector serialization error in example code of ModelSelectionViaTrainValidationSplitExample and JavaModelSelectionViaTrainValidationSplitExample

2016-02-23 Thread Xusen Yin (JIRA)
Xusen Yin created SPARK-13462:
-

 Summary: Vector serialization error in example code of 
ModelSelectionViaTrainValidationSplitExample and 
JavaModelSelectionViaTrainValidationSplitExample
 Key: SPARK-13462
 URL: https://issues.apache.org/jira/browse/SPARK-13462
 Project: Spark
  Issue Type: Sub-task
  Components: Documentation, Examples
Reporter: Xusen Yin
Priority: Minor


ModelSelectionViaTrainValidationSplitExample and 
JavaModelSelectionViaTrainValidationSplitExample fail to run. If finally it's a 
bug of TrainValidationSplit or LinearRegression, let's move the JIRA out of 
SPARK-11337.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (SPARK-13461) Duplicated example code merge and cleanup

2016-02-23 Thread Xusen Yin (JIRA)
Xusen Yin created SPARK-13461:
-

 Summary: Duplicated example code merge and cleanup
 Key: SPARK-13461
 URL: https://issues.apache.org/jira/browse/SPARK-13461
 Project: Spark
  Issue Type: Sub-task
  Components: Documentation
Reporter: Xusen Yin
Priority: Minor


Merge duplicated code after we finishing the example code substitution.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-10759) Missing Python code example in model selection user guide

2016-02-23 Thread Xusen Yin (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-10759?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xusen Yin updated SPARK-10759:
--
Issue Type: Sub-task  (was: Improvement)
Parent: SPARK-11337

> Missing Python code example in model selection user guide
> -
>
> Key: SPARK-10759
> URL: https://issues.apache.org/jira/browse/SPARK-10759
> Project: Spark
>  Issue Type: Sub-task
>  Components: Documentation, ML
>Affects Versions: 1.5.0
>Reporter: Raela Wang
>Assignee: Jeremy
>Priority: Minor
>  Labels: starter
>
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-cross-validation
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-train-validation-split



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-12922) Implement gapply() on DataFrame in SparkR

2016-02-23 Thread Narine Kokhlikyan (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12922?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159736#comment-15159736
 ] 

Narine Kokhlikyan commented on SPARK-12922:
---

Thanks for your quick response [~sunrui], I'll try to review it in detail.

> Implement gapply() on DataFrame in SparkR
> -
>
> Key: SPARK-12922
> URL: https://issues.apache.org/jira/browse/SPARK-12922
> Project: Spark
>  Issue Type: Sub-task
>  Components: SparkR
>Affects Versions: 1.6.0
>Reporter: Sun Rui
>
> gapply() applies an R function on groups grouped by one or more columns of a 
> DataFrame, and returns a DataFrame. It is like GroupedDataSet.flatMapGroups() 
> in the Dataset API.
> Two API styles are supported:
> 1.
> {code}
> gd <- groupBy(df, col1, ...)
> gapply(gd, function(grouping_key, group) {}, schema)
> {code}
> 2.
> {code}
> gapply(df, grouping_columns, function(grouping_key, group) {}, schema) 
> {code}
> R function input: grouping keys value, a local data.frame of this grouped 
> data 
> R function output: local data.frame
> Schema specifies the Row format of the output of the R function. It must 
> match the R function's output.
> Note that map-side combination (partial aggregation) is not supported, user 
> could do map-side combination via dapply().



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-13445) Seleting "data" with window function does not work unless aliased (using PARTITION BY)

2016-02-23 Thread Michael Armbrust (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13445?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Michael Armbrust updated SPARK-13445:
-
Priority: Critical  (was: Major)

> Seleting "data" with window function does not work unless aliased (using 
> PARTITION BY)
> --
>
> Key: SPARK-13445
> URL: https://issues.apache.org/jira/browse/SPARK-13445
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Reporter: Reynold Xin
>Priority: Critical
>
> The code does not throw an exception if "data" is aliased.  Maybe this is a 
> reserved word or aliases are just required when using PARTITION BY?
> {code}
> sql("""
>   SELECT 
> data as the_data,
> row_number() over (partition BY data.type) AS foo
>   FROM event_record_sample
> """)
> {code}
> However, this code throws an error:
> {code}
> sql("""
>   SELECT 
> data,
> row_number() over (partition BY data.type) AS foo
>   FROM event_record_sample
> """)
> {code}
> {code}
> org.apache.spark.sql.AnalysisException: resolved attribute(s) type#15246 
> missing from 
> data#15107,par_cat#15112,schemaMajorVersion#15110,source#15108,recordId#15103,features#15106,eventType#15105,ts#15104L,schemaMinorVersion#15111,issues#15109
>  in operator !Project [data#15107,type#15246];
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:38)
>   at 
> org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:44)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:183)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:50)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:105)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at scala.collection.immutable.List.foreach(List.scala:318)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at scala.collection.immutable.List.foreach(List.scala:318)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$foreachUp$1.apply(TreeNode.scala:104)
>   at scala.collection.immutable.List.foreach(List.scala:318)
>   at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:104)
>   at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:50)
>   at 
> org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:44)
>   at 
> org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:34)
>   at org.apache.spark.sql.DataFrame.(DataFrame.scala:133)
>   at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52)
>   at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:816)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13410) unionAll AnalysisException with DataFrames containing UDT columns.

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13410?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159719#comment-15159719
 ] 

Apache Spark commented on SPARK-13410:
--

User 'damnMeddlingKid' has created a pull request for this issue:
https://github.com/apache/spark/pull/11333

> unionAll AnalysisException with DataFrames containing UDT columns.
> --
>
> Key: SPARK-13410
> URL: https://issues.apache.org/jira/browse/SPARK-13410
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.5.0, 1.6.0
>Reporter: Franklyn Dsouza
>Assignee: Franklyn Dsouza
>  Labels: patch
> Fix For: 2.0.0
>
>   Original Estimate: 3h
>  Remaining Estimate: 3h
>
> Unioning two DataFrames that contain UDTs fails with 
> {quote}
> AnalysisException: u"unresolved operator 'Union;"
> {quote}
> I tracked this down to this line 
> https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala#L202
> Which compares datatypes between the output attributes of both logical plans. 
> However for UDTs this will be a new instance of the UserDefinedType or 
> PythonUserDefinedType 
> https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala#L158
>  
> So this equality check will check if the two instances are the same and since 
> they aren't references to a singleton this check fails. 
> *Note: this will work fine if you are unioning the dataframe with itself.*
> I have a proposed patch for this which overrides the equality operator on the 
> two classes here: https://github.com/apache/spark/pull/11279
> Reproduction steps
> {code}
> from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
> from pyspark.sql import types
> schema = types.StructType([types.StructField("point", PythonOnlyUDT(), True)])
> #note they need to be two separate dataframes
> a = sqlCtx.createDataFrame([[PythonOnlyPoint(1.0, 2.0)]], schema)
> b = sqlCtx.createDataFrame([[PythonOnlyPoint(3.0, 4.0)]], schema)
> c = a.unionAll(b)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-6761) Approximate quantile

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-6761?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159699#comment-15159699
 ] 

Apache Spark commented on SPARK-6761:
-

User 'mengxr' has created a pull request for this issue:
https://github.com/apache/spark/pull/11332

> Approximate quantile
> 
>
> Key: SPARK-6761
> URL: https://issues.apache.org/jira/browse/SPARK-6761
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Reporter: Reynold Xin
>Assignee: Liang-Chi Hsieh
> Fix For: 2.0.0
>
>
> See mailing list discussion: 
> http://apache-spark-developers-list.1001551.n3.nabble.com/Approximate-rank-based-statistics-median-95-th-percentile-etc-for-Spark-td11414.html



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13431:


Assignee: (was: Apache Spark)

> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Priority: Blocker
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Davies Liu (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159682#comment-15159682
 ] 

Davies Liu commented on SPARK-13431:


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

> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Priority: Blocker
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13431:


Assignee: Apache Spark

> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Assignee: Apache Spark
>Priority: Blocker
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159684#comment-15159684
 ] 

Apache Spark commented on SPARK-13431:
--

User 'davies' has created a pull request for this issue:
https://github.com/apache/spark/pull/11331

> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Priority: Blocker
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-5992) Locality Sensitive Hashing (LSH) for MLlib

2016-02-23 Thread Karl Higley (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-5992?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159679#comment-15159679
 ] 

Karl Higley commented on SPARK-5992:


I've been working on [a Spark package for approximate nearest 
neighbors|https://github.com/karlhigley/spark-neighbors] that implements 
several LSH flavors for different distance measures behind a unified interface. 
Currently, the package supports Hamming, Jaccard, Euclidean, and cosine 
distance. It's still a work in progress, but maybe it will provide some food 
for thought on how to proceed with the implementation for MLlib.

> Locality Sensitive Hashing (LSH) for MLlib
> --
>
> Key: SPARK-5992
> URL: https://issues.apache.org/jira/browse/SPARK-5992
> Project: Spark
>  Issue Type: New Feature
>  Components: MLlib
>Reporter: Joseph K. Bradley
>
> Locality Sensitive Hashing (LSH) would be very useful for ML.  It would be 
> great to discuss some possible algorithms here, choose an API, and make a PR 
> for an initial algorithm.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13410) unionAll AnalysisException with DataFrames containing UDT columns.

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13410?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159669#comment-15159669
 ] 

Apache Spark commented on SPARK-13410:
--

User 'damnMeddlingKid' has created a pull request for this issue:
https://github.com/apache/spark/pull/11330

> unionAll AnalysisException with DataFrames containing UDT columns.
> --
>
> Key: SPARK-13410
> URL: https://issues.apache.org/jira/browse/SPARK-13410
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.5.0, 1.6.0
>Reporter: Franklyn Dsouza
>Assignee: Franklyn Dsouza
>  Labels: patch
> Fix For: 2.0.0
>
>   Original Estimate: 3h
>  Remaining Estimate: 3h
>
> Unioning two DataFrames that contain UDTs fails with 
> {quote}
> AnalysisException: u"unresolved operator 'Union;"
> {quote}
> I tracked this down to this line 
> https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala#L202
> Which compares datatypes between the output attributes of both logical plans. 
> However for UDTs this will be a new instance of the UserDefinedType or 
> PythonUserDefinedType 
> https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala#L158
>  
> So this equality check will check if the two instances are the same and since 
> they aren't references to a singleton this check fails. 
> *Note: this will work fine if you are unioning the dataframe with itself.*
> I have a proposed patch for this which overrides the equality operator on the 
> two classes here: https://github.com/apache/spark/pull/11279
> Reproduction steps
> {code}
> from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
> from pyspark.sql import types
> schema = types.StructType([types.StructField("point", PythonOnlyUDT(), True)])
> #note they need to be two separate dataframes
> a = sqlCtx.createDataFrame([[PythonOnlyPoint(1.0, 2.0)]], schema)
> b = sqlCtx.createDataFrame([[PythonOnlyPoint(3.0, 4.0)]], schema)
> c = a.unionAll(b)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Davies Liu (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159670#comment-15159670
 ] 

Davies Liu commented on SPARK-13431:


I'd like to split ExpressionParser.g, or we can't touch it anymore (may break 
sbt break next time), unless we can switch to ANTR4 soon.

> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Priority: Blocker
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13459) Separate Alive and Dead Executors in Executor Totals Table

2016-02-23 Thread Alex Bozarth (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159657#comment-15159657
 ] 

Alex Bozarth commented on SPARK-13459:
--

That PR isn't for this Jira, I mixed up my Jira numbers while opening a PR for 
the other Jira I was working on

> Separate Alive and Dead Executors in Executor Totals Table
> --
>
> Key: SPARK-13459
> URL: https://issues.apache.org/jira/browse/SPARK-13459
> Project: Spark
>  Issue Type: Improvement
>  Components: Web UI
>Affects Versions: 2.0.0
>Reporter: Alex Bozarth
>Priority: Minor
>
> Now that dead executors are shown in the executors table (SPARK-7729) the 
> totals table added in SPARK-12716 should be updated to include the separate 
> totals for alive and dead executors as well as the current total.
> (This improvement was originally discussed in the PR for SPARK-12716 while 
> SPARK-7729 was still in progress.)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13306) Initial implementation for uncorrelated scalar subquery

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13306?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159658#comment-15159658
 ] 

Apache Spark commented on SPARK-13306:
--

User 'hvanhovell' has created a pull request for this issue:
https://github.com/apache/spark/pull/11329

> Initial implementation for uncorrelated scalar subquery
> ---
>
> Key: SPARK-13306
> URL: https://issues.apache.org/jira/browse/SPARK-13306
> Project: Spark
>  Issue Type: Sub-task
>  Components: SQL
>Reporter: Davies Liu
>Assignee: Davies Liu
> Fix For: 2.0.0
>
>
> A scalar subquery is a subquery that only generate single row and single 
> column, could be used as part of expression.
> Uncorrelated scalar subquery means it does not has a reference to external 
> table.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Herman van Hovell (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159660#comment-15159660
 ] 

Herman van Hovell commented on SPARK-13431:
---

Submitted PR for revert: https://github.com/apache/spark/pull/11329

> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Priority: Blocker
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13241) add long--formatted timestamps to org.apache.spark.status.api.v1.ApplicationAttemptInfo

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159653#comment-15159653
 ] 

Apache Spark commented on SPARK-13241:
--

User 'ajbozarth' has created a pull request for this issue:
https://github.com/apache/spark/pull/11326

> add long--formatted timestamps to 
> org.apache.spark.status.api.v1.ApplicationAttemptInfo
> ---
>
> Key: SPARK-13241
> URL: https://issues.apache.org/jira/browse/SPARK-13241
> Project: Spark
>  Issue Type: Improvement
>  Components: Web UI
>Affects Versions: 1.6.0
>Reporter: Steve Loughran
>
> While writing tests to parse 
> {{org.apache.spark.status.api.v1.ApplicationAttemptInfo}} coming off the 
> history rest server, I can see that I can't actually unmarshall the 
> timestamps without parsing the strings —and I fear that may be local specific.
> The problem here is that jersey is marshalling {{Date}} classes by calling 
> {{Date.toString()}}, leaving an entry like
> {code}
>   {
> "id": "application__",
> "name": "spark-demo",
> "attempts": [
>   {
> "attemptId": "1",
> "startTime": "2016-02-08T20:12:20.825GMT",
> "endTime": "1970-01-01T00:00:00.000GMT",
> "lastUpdated": "2016-02-08T20:12:20.848GMT",
> "duration": 0,
> "sparkUser": "hdfs",
> "completed": false
>   }
> ]
>   }
> {code}
> This is good for humans and the web UI, bad for any code trying to use the 
> API for its own purposes.
> I propose adding, alongside the existing values, a simple {{long}} value for 
> each time, representing the time in millis from the start of the epoch. This 
> is trivial to marshall/unmarshall.
> This is easy to add, provided everyone is happy that adding a new field to 
> the returned JSON is considered compatible with the existing REST API.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13241) add long--formatted timestamps to org.apache.spark.status.api.v1.ApplicationAttemptInfo

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13241?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13241:


Assignee: Apache Spark

> add long--formatted timestamps to 
> org.apache.spark.status.api.v1.ApplicationAttemptInfo
> ---
>
> Key: SPARK-13241
> URL: https://issues.apache.org/jira/browse/SPARK-13241
> Project: Spark
>  Issue Type: Improvement
>  Components: Web UI
>Affects Versions: 1.6.0
>Reporter: Steve Loughran
>Assignee: Apache Spark
>
> While writing tests to parse 
> {{org.apache.spark.status.api.v1.ApplicationAttemptInfo}} coming off the 
> history rest server, I can see that I can't actually unmarshall the 
> timestamps without parsing the strings —and I fear that may be local specific.
> The problem here is that jersey is marshalling {{Date}} classes by calling 
> {{Date.toString()}}, leaving an entry like
> {code}
>   {
> "id": "application__",
> "name": "spark-demo",
> "attempts": [
>   {
> "attemptId": "1",
> "startTime": "2016-02-08T20:12:20.825GMT",
> "endTime": "1970-01-01T00:00:00.000GMT",
> "lastUpdated": "2016-02-08T20:12:20.848GMT",
> "duration": 0,
> "sparkUser": "hdfs",
> "completed": false
>   }
> ]
>   }
> {code}
> This is good for humans and the web UI, bad for any code trying to use the 
> API for its own purposes.
> I propose adding, alongside the existing values, a simple {{long}} value for 
> each time, representing the time in millis from the start of the epoch. This 
> is trivial to marshall/unmarshall.
> This is easy to add, provided everyone is happy that adding a new field to 
> the returned JSON is considered compatible with the existing REST API.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-13459) Separate Alive and Dead Executors in Executor Totals Table

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13459:


Assignee: Apache Spark

> Separate Alive and Dead Executors in Executor Totals Table
> --
>
> Key: SPARK-13459
> URL: https://issues.apache.org/jira/browse/SPARK-13459
> Project: Spark
>  Issue Type: Improvement
>  Components: Web UI
>Affects Versions: 2.0.0
>Reporter: Alex Bozarth
>Assignee: Apache Spark
>Priority: Minor
>
> Now that dead executors are shown in the executors table (SPARK-7729) the 
> totals table added in SPARK-12716 should be updated to include the separate 
> totals for alive and dead executors as well as the current total.
> (This improvement was originally discussed in the PR for SPARK-12716 while 
> SPARK-7729 was still in progress.)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13459) Separate Alive and Dead Executors in Executor Totals Table

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159651#comment-15159651
 ] 

Apache Spark commented on SPARK-13459:
--

User 'ajbozarth' has created a pull request for this issue:
https://github.com/apache/spark/pull/11326

> Separate Alive and Dead Executors in Executor Totals Table
> --
>
> Key: SPARK-13459
> URL: https://issues.apache.org/jira/browse/SPARK-13459
> Project: Spark
>  Issue Type: Improvement
>  Components: Web UI
>Affects Versions: 2.0.0
>Reporter: Alex Bozarth
>Priority: Minor
>
> Now that dead executors are shown in the executors table (SPARK-7729) the 
> totals table added in SPARK-12716 should be updated to include the separate 
> totals for alive and dead executors as well as the current total.
> (This improvement was originally discussed in the PR for SPARK-12716 while 
> SPARK-7729 was still in progress.)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst

2016-02-23 Thread Kazuaki Ishizaki (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159649#comment-15159649
 ] 

Kazuaki Ishizaki commented on SPARK-13431:
--

I identified why this problem occurs only in maven. Shade plugin for maven 
increases the length of Java bytecode for a method. This increasing happens 
since shade plugin rewrites Java bytecode to rebuild constant pool.

Here is an output of ``javap -c SparkSqlParser_ExpressionParser.class`` before 
applying shade plugin. The static initializer ``static{}`` uses ``ldc`` 
bytecode for accessing constant pool at offset 13, 18, and 23. Each ``ldc`` 
consume only two bytes. As a result, the bytecode length of this method is 
*less than 65536*.
{code}
public class 
org.apache.spark.sql.catalyst.parser.SparkSqlParser_ExpressionParser extends 
org.antlr.runtime.Parser {
...
  static {};
Code:
   0: bipush70
   2: anewarray #1035   // class java/lang/String
   5: dup
   6: iconst_0
   7: ldc_w #1036   // String ...
  10: aastore
  11: dup
  12: iconst_1
  13: ldc   #127// String
  15: aastore
  16: dup
  17: iconst_2
  18: ldc   #127// String
  20: aastore
  21: dup
  22: iconst_3
  23: ldc   #127// String
  25: aastore
  ...
   59900: return
  }
}
{code}

After applying shade plugin, the static initializer ``static{}`` uses ``ldc_w`` 
bytecode for accessing constant pool at offset 13, 19, and 25. Each ``ldc_w`` 
consumes three bytes.  As a result, the bytecode length of this method is *more 
than 65535*.
{code}
  static {};
Code:
   0: bipush70
   2: anewarray #2965   // class java/lang/String
   5: dup
   6: iconst_0
   7: ldc_w #5240   // String ...
  10: aastore
  11: dup
  12: iconst_1
  13: ldc_w #2924   // String
  16: aastore
  17: dup
  18: iconst_2
  19: ldc_w #2924   // String
  22: aastore
  23: dup
  24: iconst_3
  25: ldc_w #2924   // String
  28: aastore
  ...
   65533: lconst_0
   65534: lastore
  ...

  }
}
{code}

Shading plugin seems to rebuild constant pool based on [this 
comment|http://svn.apache.org/viewvc/maven/plugins/tags/maven-shade-plugin-2.4.3/src/main/java/org/apache/maven/plugins/shade/DefaultShader.java?view=markup#l417].
 To use a lot of constant pool entry due to many definitions of String may 
increase the entry index of the constant pool. As a result, it leads to replace 
``ldc`` with ``ldc_w``. Finally, the length of Java bytecode is increased.

As a next step, what will we do?
* Can we avoid this rebuild by an option?
* Can we create a pull request for shade plugin to avoid this?
* Can we use another plugin?
* Can we split ExpressionParser.g into smaller files?
* Other solutions?




> Maven build fails due to: Method code too large! in Catalyst
> 
>
> Key: SPARK-13431
> URL: https://issues.apache.org/jira/browse/SPARK-13431
> Project: Spark
>  Issue Type: Bug
>  Components: Build
>Affects Versions: 2.0.0
>Reporter: Stavros Kontopoulos
>Priority: Blocker
>
> Cannot build the project when run the normal build commands:
> eg.
> {code}
> build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0  clean package
> ./make-distribution.sh --name test --tgz -Phadoop-2.6 
> {code}
> Integration builds are also failing: 
> https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console
> https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console
> It looks like this is the commit that introduced the issue:
> https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-12313) getPartitionsByFilter doesnt handle predicates on all / multiple Partition Columns

2016-02-23 Thread Apache Spark (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-12313?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159642#comment-15159642
 ] 

Apache Spark commented on SPARK-12313:
--

User 'davies' has created a pull request for this issue:
https://github.com/apache/spark/pull/11328

> getPartitionsByFilter doesnt handle predicates on all / multiple Partition 
> Columns
> --
>
> Key: SPARK-12313
> URL: https://issues.apache.org/jira/browse/SPARK-12313
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.5.1
>Reporter: Gobinathan SP
>Priority: Critical
>
> When enabled spark.sql.hive.metastorePartitionPruning, the 
> getPartitionsByFilter is used
> For a table partitioned by p1 and p2, when triggered hc.sql("select col 
> from tabl1 where p1='p1V' and p2= 'p2V' ")
> The HiveShim identifies the Predicates and ConvertFilters returns p1='p1V' 
> and col2= 'p2V' . The same is passed to the getPartitionsByFilter method as 
> filter string.
> On these cases the partitions are not returned from Hive's 
> getPartitionsByFilter method. As a result, for the sql, the number of 
> returned rows is always zero. 
> However, filter on a single column always works. Probalbly  it doesn't come 
> through this route
> I'm using Oracle for Metstore V0.13.1



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-12313) getPartitionsByFilter doesnt handle predicates on all / multiple Partition Columns

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-12313?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-12313:


Assignee: (was: Apache Spark)

> getPartitionsByFilter doesnt handle predicates on all / multiple Partition 
> Columns
> --
>
> Key: SPARK-12313
> URL: https://issues.apache.org/jira/browse/SPARK-12313
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.5.1
>Reporter: Gobinathan SP
>Priority: Critical
>
> When enabled spark.sql.hive.metastorePartitionPruning, the 
> getPartitionsByFilter is used
> For a table partitioned by p1 and p2, when triggered hc.sql("select col 
> from tabl1 where p1='p1V' and p2= 'p2V' ")
> The HiveShim identifies the Predicates and ConvertFilters returns p1='p1V' 
> and col2= 'p2V' . The same is passed to the getPartitionsByFilter method as 
> filter string.
> On these cases the partitions are not returned from Hive's 
> getPartitionsByFilter method. As a result, for the sql, the number of 
> returned rows is always zero. 
> However, filter on a single column always works. Probalbly  it doesn't come 
> through this route
> I'm using Oracle for Metstore V0.13.1



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Assigned] (SPARK-12313) getPartitionsByFilter doesnt handle predicates on all / multiple Partition Columns

2016-02-23 Thread Apache Spark (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-12313?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-12313:


Assignee: Apache Spark

> getPartitionsByFilter doesnt handle predicates on all / multiple Partition 
> Columns
> --
>
> Key: SPARK-12313
> URL: https://issues.apache.org/jira/browse/SPARK-12313
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.5.1
>Reporter: Gobinathan SP
>Assignee: Apache Spark
>Priority: Critical
>
> When enabled spark.sql.hive.metastorePartitionPruning, the 
> getPartitionsByFilter is used
> For a table partitioned by p1 and p2, when triggered hc.sql("select col 
> from tabl1 where p1='p1V' and p2= 'p2V' ")
> The HiveShim identifies the Predicates and ConvertFilters returns p1='p1V' 
> and col2= 'p2V' . The same is passed to the getPartitionsByFilter method as 
> filter string.
> On these cases the partitions are not returned from Hive's 
> getPartitionsByFilter method. As a result, for the sql, the number of 
> returned rows is always zero. 
> However, filter on a single column always works. Probalbly  it doesn't come 
> through this route
> I'm using Oracle for Metstore V0.13.1



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13460) Applying Encoding methods to GraphX's Internal storage structure

2016-02-23 Thread Adnan Haider (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159641#comment-15159641
 ] 

Adnan Haider commented on SPARK-13460:
--

I am currently working on submitting a pull request for run-length encoding 
local source ids. 

> Applying Encoding methods to GraphX's Internal storage structure
> 
>
> Key: SPARK-13460
> URL: https://issues.apache.org/jira/browse/SPARK-13460
> Project: Spark
>  Issue Type: Improvement
>  Components: GraphX
>Reporter: Adnan Haider
>   Original Estimate: 336h
>  Remaining Estimate: 336h
>
> Currently, the memory consumption of graphs is more than 2x the graph's 
> on-disk size. For example, loading the Friendster dataset consumes 32.3 GB on 
> disk but 85.1 GB after loading it into GraphX. Some of this memory 
> consumption can be reduced by using [delta 
> encoding|https://en.wikipedia.org/wiki/Delta_encoding] and [run-length 
> encoding|https://en.wikipedia.org/wiki/Run-length_encoding]. 
> For example, we can use run-length encoding on the localSrcIds. Instead, of 
> storing a local source id for each edge, we can store a map from each local 
> source id to a tuple of offset, length pairs, which index into localDstIds:
> {code:title=EdgePartitionBuilder.scala}
> val index = new GraphXPrimitiveKeyOpenHashMap[Int, (Int, Int)]
> {code}
> This method saves about 17% of space for the [Friendster 
> dataset|https://snap.stanford.edu/data/com-Friendster.html] or around 14 GB. 
> It also saves 14% for the [orkut 
> dataset|https://snap.stanford.edu/data/com-Orkut.html].
> In addition, we could use delta encoding on the localDstIds. After sorting 
> the local destination ids for each source vertex and then apply the encoding, 
> for Friendster half of the ids can be represented in a single byte.
>  
> Also, currently GraphX stores multiple copies of global vertex ids for 
> different mapping structures. Depending on the graph partitioning strategy 
> and how the edge list file is stored on disk, the global vertex ids can be 
> close to each other, meaning they are a good candidate for delta encoding. 
> Collectively these features provide around 50% savings for the Friendster 
> dataset. These savings come from a prototype version I have worked on. Some 
> of these savings requires a great deal of modification to internal GraphX 
> code, since algorithms need to be changed to limit the overhead of decoding 
> data. For example, below is a snippet from the modified filter operation in 
> EdgePartition.scala:
> {code:title=Filter}
> index.iterator.foreach { cluster =>
>val clusterLocalSrcId = cluster._1
>val dstOffset = cluster._2._1
>val dstLen = cluster._2._2
>   // The user sees the EdgeTriplet, so we can't reuse it and must create 
> one per edge.
>   val localSrcId = clusterLocalSrcId
>   var i = 0
>   while(i < dstLen){
> val localDstId = localDstIds(i + dstOffset)
> {code}
> This can be a long term design change and will help address some recent 
> [results|http://www.vldb.org/pvldb/vol8/p161-bu.pdf] that GraphX has trouble 
> loading large datasets.  I have a discussion here on the [dev 
> list|http://apache-spark-developers-list.1001551.n3.nabble.com/Using-Encoding-to-reduce-GraphX-s-static-graph-memory-consumption-tp16373.html]
>  where we can discuss what encoding features are worth adding. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Commented] (SPARK-13307) TPCDS query 66 degraded by 30% in 1.6.0 compared to 1.4.1

2016-02-23 Thread JESSE CHEN (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-13307?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159636#comment-15159636
 ] 

JESSE CHEN commented on SPARK-13307:


I tuned up the autoBroadcastJoinThreshold to 100MB, made no difference:
spark.sql.autoBroadcastJoinThreshold 104857600
query time is 913 seconds.

Now, how would you change the query to add the [broadcast] hint? 

I will try it if you provide me the modified query.

Thanks.


> TPCDS query 66 degraded by 30% in 1.6.0 compared to 1.4.1
> -
>
> Key: SPARK-13307
> URL: https://issues.apache.org/jira/browse/SPARK-13307
> Project: Spark
>  Issue Type: Bug
>  Components: SQL
>Affects Versions: 1.6.0
>Reporter: JESSE CHEN
>
> Majority of the TPCDS queries ran faster in 1.6.0 than in 1.4.1, average 
> about 9% faster. There are a few degraded, and one that is definitely not 
> within error margin is query 66.
> Query 66 in 1.4.1: 699 seconds
> Query 66 in 1.6.0: 918 seconds
> 30% worse.
> Collected the physical plans from both versions - drastic difference maybe 
> partially from using Tungsten in 1.6, but anything else at play here?
> Please see plans here:
> https://ibm.box.com/spark-sql-q66-debug-160plan
> https://ibm.box.com/spark-sql-q66-debug-141plan



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Resolved] (SPARK-13329) Considering output for statistics of logical plan

2016-02-23 Thread Davies Liu (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-13329?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Davies Liu resolved SPARK-13329.

   Resolution: Fixed
Fix Version/s: 2.0.0

Issue resolved by pull request 11210
[https://github.com/apache/spark/pull/11210]

> Considering output for statistics of logical plan
> -
>
> Key: SPARK-13329
> URL: https://issues.apache.org/jira/browse/SPARK-13329
> Project: Spark
>  Issue Type: Improvement
>  Components: SQL
>Reporter: Davies Liu
>Assignee: Davies Liu
> Fix For: 2.0.0
>
>
> The current implementation of statistics of UnaryNode does not considering 
> output (for example, Project), we should considering it to have a better 
> guess.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-11381) Replace example code in mllib-linear-methods.md using include_example

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-11381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-11381:
--
Target Version/s: 2.0.0
 Component/s: MLlib

> Replace example code in mllib-linear-methods.md using include_example
> -
>
> Key: SPARK-11381
> URL: https://issues.apache.org/jira/browse/SPARK-11381
> Project: Spark
>  Issue Type: Sub-task
>  Components: Documentation, MLlib
>Reporter: Xusen Yin
>Assignee: Dongjoon Hyun
>  Labels: starter
>
> This is similar to SPARK-11289 but for the example code in 
> mllib-linear-methods.md.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-11381) Replace example code in mllib-linear-methods.md using include_example

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-11381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-11381:
--
Assignee: Dongjoon Hyun

> Replace example code in mllib-linear-methods.md using include_example
> -
>
> Key: SPARK-11381
> URL: https://issues.apache.org/jira/browse/SPARK-11381
> Project: Spark
>  Issue Type: Sub-task
>  Components: Documentation, MLlib
>Reporter: Xusen Yin
>Assignee: Dongjoon Hyun
>  Labels: starter
>
> This is similar to SPARK-11289 but for the example code in 
> mllib-linear-methods.md.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-10759) Missing Python code example in model selection user guide

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-10759?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-10759:
--
Shepherd: Xusen Yin
Assignee: Jeremy  (was: Apache Spark)

> Missing Python code example in model selection user guide
> -
>
> Key: SPARK-10759
> URL: https://issues.apache.org/jira/browse/SPARK-10759
> Project: Spark
>  Issue Type: Improvement
>  Components: Documentation, ML
>Affects Versions: 1.5.0
>Reporter: Raela Wang
>Assignee: Jeremy
>Priority: Minor
>  Labels: starter
>
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-cross-validation
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-train-validation-split



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-10759) Missing Python code example in model selection user guide

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-10759?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-10759:
--
Target Version/s: 2.0.0
 Component/s: ML

> Missing Python code example in model selection user guide
> -
>
> Key: SPARK-10759
> URL: https://issues.apache.org/jira/browse/SPARK-10759
> Project: Spark
>  Issue Type: Improvement
>  Components: Documentation, ML
>Affects Versions: 1.5.0
>Reporter: Raela Wang
>Assignee: Apache Spark
>Priority: Minor
>  Labels: starter
>
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-cross-validation
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-train-validation-split



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-10759) Missing Python code example in model selection user guide

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-10759?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-10759:
--
Summary: Missing Python code example in model selection user guide  (was: 
Missing Python code example in ML Programming guide)

> Missing Python code example in model selection user guide
> -
>
> Key: SPARK-10759
> URL: https://issues.apache.org/jira/browse/SPARK-10759
> Project: Spark
>  Issue Type: Improvement
>  Components: Documentation, ML
>Affects Versions: 1.5.0
>Reporter: Raela Wang
>Assignee: Apache Spark
>Priority: Minor
>  Labels: starter
>
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-cross-validation
> http://spark.apache.org/docs/latest/ml-guide.html#example-model-selection-via-train-validation-split



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Created] (SPARK-13460) Applying Encoding methods to GraphX's Internal storage structure

2016-02-23 Thread Adnan Haider (JIRA)
Adnan Haider created SPARK-13460:


 Summary: Applying Encoding methods to GraphX's Internal storage 
structure
 Key: SPARK-13460
 URL: https://issues.apache.org/jira/browse/SPARK-13460
 Project: Spark
  Issue Type: Improvement
  Components: GraphX
Reporter: Adnan Haider


Currently, the memory consumption of graphs is more than 2x the graph's on-disk 
size. For example, loading the Friendster dataset consumes 32.3 GB on disk but 
85.1 GB after loading it into GraphX. Some of this memory consumption can be 
reduced by using [delta encoding|https://en.wikipedia.org/wiki/Delta_encoding] 
and [run-length encoding|https://en.wikipedia.org/wiki/Run-length_encoding]. 


For example, we can use run-length encoding on the localSrcIds. Instead, of 
storing a local source id for each edge, we can store a map from each local 
source id to a tuple of offset, length pairs, which index into localDstIds:
{code:title=EdgePartitionBuilder.scala}
val index = new GraphXPrimitiveKeyOpenHashMap[Int, (Int, Int)]
{code}

This method saves about 17% of space for the [Friendster 
dataset|https://snap.stanford.edu/data/com-Friendster.html] or around 14 GB. It 
also saves 14% for the [orkut 
dataset|https://snap.stanford.edu/data/com-Orkut.html].


In addition, we could use delta encoding on the localDstIds. After sorting the 
local destination ids for each source vertex and then apply the encoding, for 
Friendster half of the ids can be represented in a single byte.
 
Also, currently GraphX stores multiple copies of global vertex ids for 
different mapping structures. Depending on the graph partitioning strategy and 
how the edge list file is stored on disk, the global vertex ids can be close to 
each other, meaning they are a good candidate for delta encoding. 


Collectively these features provide around 50% savings for the Friendster 
dataset. These savings come from a prototype version I have worked on. Some of 
these savings requires a great deal of modification to internal GraphX code, 
since algorithms need to be changed to limit the overhead of decoding data. For 
example, below is a snippet from the modified filter operation in 
EdgePartition.scala:
{code:title=Filter}
index.iterator.foreach { cluster =>
   val clusterLocalSrcId = cluster._1
   val dstOffset = cluster._2._1
   val dstLen = cluster._2._2
  // The user sees the EdgeTriplet, so we can't reuse it and must create 
one per edge.
  val localSrcId = clusterLocalSrcId
  var i = 0
  while(i < dstLen){
val localDstId = localDstIds(i + dstOffset)
{code}

This can be a long term design change and will help address some recent 
[results|http://www.vldb.org/pvldb/vol8/p161-bu.pdf] that GraphX has trouble 
loading large datasets.  I have a discussion here on the [dev 
list|http://apache-spark-developers-list.1001551.n3.nabble.com/Using-Encoding-to-reduce-GraphX-s-static-graph-memory-consumption-tp16373.html]
 where we can discuss what encoding features are worth adding. 




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-7106) Support model save/load in Python's FPGrowth

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-7106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-7106:
-
Target Version/s: 2.0.0

> Support model save/load in Python's FPGrowth
> 
>
> Key: SPARK-7106
> URL: https://issues.apache.org/jira/browse/SPARK-7106
> Project: Spark
>  Issue Type: Sub-task
>  Components: MLlib, PySpark
>Reporter: Joseph K. Bradley
>Assignee: Kai Jiang
>Priority: Minor
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-7106) Support model save/load in Python's FPGrowth

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-7106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-7106:
-
Shepherd: Xiangrui Meng  (was: Yanbo Liang)

> Support model save/load in Python's FPGrowth
> 
>
> Key: SPARK-7106
> URL: https://issues.apache.org/jira/browse/SPARK-7106
> Project: Spark
>  Issue Type: Sub-task
>  Components: MLlib, PySpark
>Reporter: Joseph K. Bradley
>Assignee: Kai Jiang
>Priority: Minor
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-7106) Support model save/load in Python's FPGrowth

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-7106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-7106:
-
Assignee: Kai Jiang

> Support model save/load in Python's FPGrowth
> 
>
> Key: SPARK-7106
> URL: https://issues.apache.org/jira/browse/SPARK-7106
> Project: Spark
>  Issue Type: Sub-task
>  Components: MLlib, PySpark
>Reporter: Joseph K. Bradley
>Assignee: Kai Jiang
>Priority: Minor
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



[jira] [Updated] (SPARK-7106) Support model save/load in Python's FPGrowth

2016-02-23 Thread Xiangrui Meng (JIRA)

 [ 
https://issues.apache.org/jira/browse/SPARK-7106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Xiangrui Meng updated SPARK-7106:
-
Shepherd: Yanbo Liang

> Support model save/load in Python's FPGrowth
> 
>
> Key: SPARK-7106
> URL: https://issues.apache.org/jira/browse/SPARK-7106
> Project: Spark
>  Issue Type: Sub-task
>  Components: MLlib, PySpark
>Reporter: Joseph K. Bradley
>Assignee: Kai Jiang
>Priority: Minor
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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



  1   2   3   >