Re: [VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-20 Thread Henry Saputra
Signature and hash for source looks good
No external executable package with source - good
Compiled with git and maven - good
Ran examples and sample programs locally and standalone -good

+1

- Henry



On Tue, May 20, 2014 at 1:13 PM, Tathagata Das
 wrote:
> Please vote on releasing the following candidate as Apache Spark version 
> 1.0.0!
>
> This has a few bug fixes on top of rc9:
> SPARK-1875: https://github.com/apache/spark/pull/824
> SPARK-1876: https://github.com/apache/spark/pull/819
> SPARK-1878: https://github.com/apache/spark/pull/822
> SPARK-1879: https://github.com/apache/spark/pull/823
>
> The tag to be voted on is v1.0.0-rc10 (commit d8070234):
> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=d807023479ce10aec28ef3c1ab646ddefc2e663c
>
> The release files, including signatures, digests, etc. can be found at:
> http://people.apache.org/~tdas/spark-1.0.0-rc10/
>
> The release artifacts are signed with the following key:
> https://people.apache.org/keys/committer/tdas.asc
>
> The staging repository for this release can be found at:
> https://repository.apache.org/content/repositories/orgapachespark-1018/
>
> The documentation corresponding to this release can be found at:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/
>
> The full list of changes in this release can be found at:
> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=blob;f=CHANGES.txt;h=d21f0ace6326e099360975002797eb7cba9d5273;hb=d807023479ce10aec28ef3c1ab646ddefc2e663c
>
> Please vote on releasing this package as Apache Spark 1.0.0!
>
> The vote is open until Friday, May 23, at 20:00 UTC and passes if
> amajority of at least 3 +1 PMC votes are cast.
>
> [ ] +1 Release this package as Apache Spark 1.0.0
> [ ] -1 Do not release this package because ...
>
> To learn more about Apache Spark, please see
> http://spark.apache.org/
>
> == API Changes ==
> We welcome users to compile Spark applications against 1.0. There are
> a few API changes in this release. Here are links to the associated
> upgrade guides - user facing changes have been kept as small as
> possible.
>
> Changes to ML vector specification:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/mllib-guide.html#from-09-to-10
>
> Changes to the Java API:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/java-programming-guide.html#upgrading-from-pre-10-versions-of-spark
>
> Changes to the streaming API:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x
>
> Changes to the GraphX API:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/graphx-programming-guide.html#upgrade-guide-from-spark-091
>
> Other changes:
> coGroup and related functions now return Iterable[T] instead of Seq[T]
> ==> Call toSeq on the result to restore the old behavior
>
> SparkContext.jarOfClass returns Option[String] instead of Seq[String]
> ==> Call toSeq on the result to restore old behavior


Re: [VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-20 Thread Matei Zaharia
+1

Tested it on both Windows and Mac OS X, with both Scala and Python. Confirmed 
that the issues in the previous RC were fixed.

Matei

On May 20, 2014, at 5:28 PM, Marcelo Vanzin  wrote:

> +1 (non-binding)
> 
> I have:
> - checked signatures and checksums of the files
> - built the code from the git repo using both sbt and mvn (against hadoop 
> 2.3.0)
> - ran a few simple jobs in local, yarn-client and yarn-cluster mode
> 
> Haven't explicitly tested any of the recent fixes, streaming nor sql.
> 
> 
> On Tue, May 20, 2014 at 1:13 PM, Tathagata Das
>  wrote:
>> Please vote on releasing the following candidate as Apache Spark version 
>> 1.0.0!
>> 
>> This has a few bug fixes on top of rc9:
>> SPARK-1875: https://github.com/apache/spark/pull/824
>> SPARK-1876: https://github.com/apache/spark/pull/819
>> SPARK-1878: https://github.com/apache/spark/pull/822
>> SPARK-1879: https://github.com/apache/spark/pull/823
>> 
>> The tag to be voted on is v1.0.0-rc10 (commit d8070234):
>> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=d807023479ce10aec28ef3c1ab646ddefc2e663c
>> 
>> The release files, including signatures, digests, etc. can be found at:
>> http://people.apache.org/~tdas/spark-1.0.0-rc10/
>> 
>> The release artifacts are signed with the following key:
>> https://people.apache.org/keys/committer/tdas.asc
>> 
>> The staging repository for this release can be found at:
>> https://repository.apache.org/content/repositories/orgapachespark-1018/
>> 
>> The documentation corresponding to this release can be found at:
>> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/
>> 
>> The full list of changes in this release can be found at:
>> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=blob;f=CHANGES.txt;h=d21f0ace6326e099360975002797eb7cba9d5273;hb=d807023479ce10aec28ef3c1ab646ddefc2e663c
>> 
>> Please vote on releasing this package as Apache Spark 1.0.0!
>> 
>> The vote is open until Friday, May 23, at 20:00 UTC and passes if
>> amajority of at least 3 +1 PMC votes are cast.
>> 
>> [ ] +1 Release this package as Apache Spark 1.0.0
>> [ ] -1 Do not release this package because ...
>> 
>> To learn more about Apache Spark, please see
>> http://spark.apache.org/
>> 
>> == API Changes ==
>> We welcome users to compile Spark applications against 1.0. There are
>> a few API changes in this release. Here are links to the associated
>> upgrade guides - user facing changes have been kept as small as
>> possible.
>> 
>> Changes to ML vector specification:
>> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/mllib-guide.html#from-09-to-10
>> 
>> Changes to the Java API:
>> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/java-programming-guide.html#upgrading-from-pre-10-versions-of-spark
>> 
>> Changes to the streaming API:
>> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x
>> 
>> Changes to the GraphX API:
>> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/graphx-programming-guide.html#upgrade-guide-from-spark-091
>> 
>> Other changes:
>> coGroup and related functions now return Iterable[T] instead of Seq[T]
>> ==> Call toSeq on the result to restore the old behavior
>> 
>> SparkContext.jarOfClass returns Option[String] instead of Seq[String]
>> ==> Call toSeq on the result to restore old behavior
> 
> 
> 
> -- 
> Marcelo



Re: Calling external classes added by sc.addJar needs to be through reflection

2014-05-20 Thread Xiangrui Meng
Talked with Sandy and DB offline. I think the best solution is sending
the secondary jars to the distributed cache of all containers rather
than just the master, and set the classpath to include spark jar,
primary app jar, and secondary jars before executor starts. In this
way, user only needs to specify secondary jars via --jars instead of
calling sc.addJar inside the code. It also solves the scalability
problem of serving all the jars via http.

If this solution sounds good, I can try to make a patch.

Best,
Xiangrui

On Mon, May 19, 2014 at 10:04 PM, DB Tsai  wrote:
> In 1.0, there is a new option for users to choose which classloader has
> higher priority via spark.files.userClassPathFirst, I decided to submit the
> PR for 0.9 first. We use this patch in our lab and we can use those jars
> added by sc.addJar without reflection.
>
> https://github.com/apache/spark/pull/834
>
> Can anyone comment if it's a good approach?
>
> Thanks.
>
>
> Sincerely,
>
> DB Tsai
> ---
> My Blog: https://www.dbtsai.com
> LinkedIn: https://www.linkedin.com/in/dbtsai
>
>
> On Mon, May 19, 2014 at 7:42 PM, DB Tsai  wrote:
>
>> Good summary! We fixed it in branch 0.9 since our production is still in
>> 0.9. I'm porting it to 1.0 now, and hopefully will submit PR for 1.0
>> tonight.
>>
>>
>> Sincerely,
>>
>> DB Tsai
>> ---
>> My Blog: https://www.dbtsai.com
>> LinkedIn: https://www.linkedin.com/in/dbtsai
>>
>>
>> On Mon, May 19, 2014 at 7:38 PM, Sandy Ryza wrote:
>>
>>> It just hit me why this problem is showing up on YARN and not on
>>> standalone.
>>>
>>> The relevant difference between YARN and standalone is that, on YARN, the
>>> app jar is loaded by the system classloader instead of Spark's custom URL
>>> classloader.
>>>
>>> On YARN, the system classloader knows about [the classes in the spark
>>> jars,
>>> the classes in the primary app jar].   The custom classloader knows about
>>> [the classes in secondary app jars] and has the system classloader as its
>>> parent.
>>>
>>> A few relevant facts (mostly redundant with what Sean pointed out):
>>> * Every class has a classloader that loaded it.
>>> * When an object of class B is instantiated inside of class A, the
>>> classloader used for loading B is the classloader that was used for
>>> loading
>>> A.
>>> * When a classloader fails to load a class, it lets its parent classloader
>>> try.  If its parent succeeds, its parent becomes the "classloader that
>>> loaded it".
>>>
>>> So suppose class B is in a secondary app jar and class A is in the primary
>>> app jar:
>>> 1. The custom classloader will try to load class A.
>>> 2. It will fail, because it only knows about the secondary jars.
>>> 3. It will delegate to its parent, the system classloader.
>>> 4. The system classloader will succeed, because it knows about the primary
>>> app jar.
>>> 5. A's classloader will be the system classloader.
>>> 6. A tries to instantiate an instance of class B.
>>> 7. B will be loaded with A's classloader, which is the system classloader.
>>> 8. Loading B will fail, because A's classloader, which is the system
>>> classloader, doesn't know about the secondary app jars.
>>>
>>> In Spark standalone, A and B are both loaded by the custom classloader, so
>>> this issue doesn't come up.
>>>
>>> -Sandy
>>>
>>> On Mon, May 19, 2014 at 7:07 PM, Patrick Wendell 
>>> wrote:
>>>
>>> > Having a user add define a custom class inside of an added jar and
>>> > instantiate it directly inside of an executor is definitely supported
>>> > in Spark and has been for a really long time (several years). This is
>>> > something we do all the time in Spark.
>>> >
>>> > DB - I'd hold off on a re-architecting of this until we identify
>>> > exactly what is causing the bug you are running into.
>>> >
>>> > In a nutshell, when the bytecode "new Foo()" is run on the executor,
>>> > it will ask the driver for the class over HTTP using a custom
>>> > classloader. Something in that pipeline is breaking here, possibly
>>> > related to the YARN deployment stuff.
>>> >
>>> >
>>> > On Mon, May 19, 2014 at 12:29 AM, Sean Owen  wrote:
>>> > > I don't think a customer classloader is necessary.
>>> > >
>>> > > Well, it occurs to me that this is no new problem. Hadoop, Tomcat, etc
>>> > > all run custom user code that creates new user objects without
>>> > > reflection. I should go see how that's done. Maybe it's totally valid
>>> > > to set the thread's context classloader for just this purpose, and I
>>> > > am not thinking clearly.
>>> > >
>>> > > On Mon, May 19, 2014 at 8:26 AM, Andrew Ash 
>>> > wrote:
>>> > >> Sounds like the problem is that classloaders always look in their
>>> > parents
>>> > >> before themselves, and Spark users want executors to pick up classes
>>> > from
>>> > >> their custom code before the ones in Spark plus its dependencies.
>>> > >>
>>> > >> Would a custom classloader that delegates to the parent

Re: [VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-20 Thread Sandy Ryza
+1


On Tue, May 20, 2014 at 5:26 PM, Andrew Or  wrote:

> +1
>
>
> 2014-05-20 13:13 GMT-07:00 Tathagata Das :
>
> > Please vote on releasing the following candidate as Apache Spark version
> > 1.0.0!
> >
> > This has a few bug fixes on top of rc9:
> > SPARK-1875: https://github.com/apache/spark/pull/824
> > SPARK-1876: https://github.com/apache/spark/pull/819
> > SPARK-1878: https://github.com/apache/spark/pull/822
> > SPARK-1879: https://github.com/apache/spark/pull/823
> >
> > The tag to be voted on is v1.0.0-rc10 (commit d8070234):
> >
> >
> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=d807023479ce10aec28ef3c1ab646ddefc2e663c
> >
> > The release files, including signatures, digests, etc. can be found at:
> > http://people.apache.org/~tdas/spark-1.0.0-rc10/
> >
> > The release artifacts are signed with the following key:
> > https://people.apache.org/keys/committer/tdas.asc
> >
> > The staging repository for this release can be found at:
> > https://repository.apache.org/content/repositories/orgapachespark-1018/
> >
> > The documentation corresponding to this release can be found at:
> > http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/
> >
> > The full list of changes in this release can be found at:
> >
> >
> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=blob;f=CHANGES.txt;h=d21f0ace6326e099360975002797eb7cba9d5273;hb=d807023479ce10aec28ef3c1ab646ddefc2e663c
> >
> > Please vote on releasing this package as Apache Spark 1.0.0!
> >
> > The vote is open until Friday, May 23, at 20:00 UTC and passes if
> > amajority of at least 3 +1 PMC votes are cast.
> >
> > [ ] +1 Release this package as Apache Spark 1.0.0
> > [ ] -1 Do not release this package because ...
> >
> > To learn more about Apache Spark, please see
> > http://spark.apache.org/
> >
> > == API Changes ==
> > We welcome users to compile Spark applications against 1.0. There are
> > a few API changes in this release. Here are links to the associated
> > upgrade guides - user facing changes have been kept as small as
> > possible.
> >
> > Changes to ML vector specification:
> >
> >
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/mllib-guide.html#from-09-to-10
> >
> > Changes to the Java API:
> >
> >
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/java-programming-guide.html#upgrading-from-pre-10-versions-of-spark
> >
> > Changes to the streaming API:
> >
> >
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x
> >
> > Changes to the GraphX API:
> >
> >
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/graphx-programming-guide.html#upgrade-guide-from-spark-091
> >
> > Other changes:
> > coGroup and related functions now return Iterable[T] instead of Seq[T]
> > ==> Call toSeq on the result to restore the old behavior
> >
> > SparkContext.jarOfClass returns Option[String] instead of Seq[String]
> > ==> Call toSeq on the result to restore old behavior
> >
>


Re: [VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-20 Thread Marcelo Vanzin
+1 (non-binding)

I have:
- checked signatures and checksums of the files
- built the code from the git repo using both sbt and mvn (against hadoop 2.3.0)
- ran a few simple jobs in local, yarn-client and yarn-cluster mode

Haven't explicitly tested any of the recent fixes, streaming nor sql.


On Tue, May 20, 2014 at 1:13 PM, Tathagata Das
 wrote:
> Please vote on releasing the following candidate as Apache Spark version 
> 1.0.0!
>
> This has a few bug fixes on top of rc9:
> SPARK-1875: https://github.com/apache/spark/pull/824
> SPARK-1876: https://github.com/apache/spark/pull/819
> SPARK-1878: https://github.com/apache/spark/pull/822
> SPARK-1879: https://github.com/apache/spark/pull/823
>
> The tag to be voted on is v1.0.0-rc10 (commit d8070234):
> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=d807023479ce10aec28ef3c1ab646ddefc2e663c
>
> The release files, including signatures, digests, etc. can be found at:
> http://people.apache.org/~tdas/spark-1.0.0-rc10/
>
> The release artifacts are signed with the following key:
> https://people.apache.org/keys/committer/tdas.asc
>
> The staging repository for this release can be found at:
> https://repository.apache.org/content/repositories/orgapachespark-1018/
>
> The documentation corresponding to this release can be found at:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/
>
> The full list of changes in this release can be found at:
> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=blob;f=CHANGES.txt;h=d21f0ace6326e099360975002797eb7cba9d5273;hb=d807023479ce10aec28ef3c1ab646ddefc2e663c
>
> Please vote on releasing this package as Apache Spark 1.0.0!
>
> The vote is open until Friday, May 23, at 20:00 UTC and passes if
> amajority of at least 3 +1 PMC votes are cast.
>
> [ ] +1 Release this package as Apache Spark 1.0.0
> [ ] -1 Do not release this package because ...
>
> To learn more about Apache Spark, please see
> http://spark.apache.org/
>
> == API Changes ==
> We welcome users to compile Spark applications against 1.0. There are
> a few API changes in this release. Here are links to the associated
> upgrade guides - user facing changes have been kept as small as
> possible.
>
> Changes to ML vector specification:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/mllib-guide.html#from-09-to-10
>
> Changes to the Java API:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/java-programming-guide.html#upgrading-from-pre-10-versions-of-spark
>
> Changes to the streaming API:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x
>
> Changes to the GraphX API:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/graphx-programming-guide.html#upgrade-guide-from-spark-091
>
> Other changes:
> coGroup and related functions now return Iterable[T] instead of Seq[T]
> ==> Call toSeq on the result to restore the old behavior
>
> SparkContext.jarOfClass returns Option[String] instead of Seq[String]
> ==> Call toSeq on the result to restore old behavior



-- 
Marcelo


Re: [VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-20 Thread Andrew Or
+1


2014-05-20 13:13 GMT-07:00 Tathagata Das :

> Please vote on releasing the following candidate as Apache Spark version
> 1.0.0!
>
> This has a few bug fixes on top of rc9:
> SPARK-1875: https://github.com/apache/spark/pull/824
> SPARK-1876: https://github.com/apache/spark/pull/819
> SPARK-1878: https://github.com/apache/spark/pull/822
> SPARK-1879: https://github.com/apache/spark/pull/823
>
> The tag to be voted on is v1.0.0-rc10 (commit d8070234):
>
> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=d807023479ce10aec28ef3c1ab646ddefc2e663c
>
> The release files, including signatures, digests, etc. can be found at:
> http://people.apache.org/~tdas/spark-1.0.0-rc10/
>
> The release artifacts are signed with the following key:
> https://people.apache.org/keys/committer/tdas.asc
>
> The staging repository for this release can be found at:
> https://repository.apache.org/content/repositories/orgapachespark-1018/
>
> The documentation corresponding to this release can be found at:
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/
>
> The full list of changes in this release can be found at:
>
> https://git-wip-us.apache.org/repos/asf?p=spark.git;a=blob;f=CHANGES.txt;h=d21f0ace6326e099360975002797eb7cba9d5273;hb=d807023479ce10aec28ef3c1ab646ddefc2e663c
>
> Please vote on releasing this package as Apache Spark 1.0.0!
>
> The vote is open until Friday, May 23, at 20:00 UTC and passes if
> amajority of at least 3 +1 PMC votes are cast.
>
> [ ] +1 Release this package as Apache Spark 1.0.0
> [ ] -1 Do not release this package because ...
>
> To learn more about Apache Spark, please see
> http://spark.apache.org/
>
> == API Changes ==
> We welcome users to compile Spark applications against 1.0. There are
> a few API changes in this release. Here are links to the associated
> upgrade guides - user facing changes have been kept as small as
> possible.
>
> Changes to ML vector specification:
>
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/mllib-guide.html#from-09-to-10
>
> Changes to the Java API:
>
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/java-programming-guide.html#upgrading-from-pre-10-versions-of-spark
>
> Changes to the streaming API:
>
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x
>
> Changes to the GraphX API:
>
> http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/graphx-programming-guide.html#upgrade-guide-from-spark-091
>
> Other changes:
> coGroup and related functions now return Iterable[T] instead of Seq[T]
> ==> Call toSeq on the result to restore the old behavior
>
> SparkContext.jarOfClass returns Option[String] instead of Seq[String]
> ==> Call toSeq on the result to restore old behavior
>


Re: Scala examples for Spark do not work as written in documentation

2014-05-20 Thread Andy Konwinski
I fixed the bug, but I kept the parameter "i" instead of "_" since that (1)
keeps it more parallel to the python and java versions which also use
functions with a named variable and (2) doesn't require readers to know
this particular use of the "_" syntax in Scala.

Thanks for catching this Glenn.

Andy


On Fri, May 16, 2014 at 12:38 PM, Mark Hamstra wrote:

> Sorry, looks like an extra line got inserted in there.  One more try:
>
> val count = spark.parallelize(1 to NUM_SAMPLES).map { _ =>
>   val x = Math.random()
>   val y = Math.random()
>   if (x*x + y*y < 1) 1 else 0
> }.reduce(_ + _)
>
>
>
> On Fri, May 16, 2014 at 12:36 PM, Mark Hamstra  >wrote:
>
> > Actually, the better way to write the multi-line closure would be:
> >
> > val count = spark.parallelize(1 to NUM_SAMPLES).map { _ =>
> >
> >   val x = Math.random()
> >   val y = Math.random()
> >   if (x*x + y*y < 1) 1 else 0
> > }.reduce(_ + _)
> >
> >
> > On Fri, May 16, 2014 at 9:41 AM, GlennStrycker  >wrote:
> >
> >> On the webpage http://spark.apache.org/examples.html, there is an
> example
> >> written as
> >>
> >> val count = spark.parallelize(1 to NUM_SAMPLES).map(i =>
> >>   val x = Math.random()
> >>   val y = Math.random()
> >>   if (x*x + y*y < 1) 1 else 0
> >> ).reduce(_ + _)
> >> println("Pi is roughly " + 4.0 * count / NUM_SAMPLES)
> >>
> >> This does not execute in Spark, which gives me an error:
> >> :2: error: illegal start of simple expression
> >>  val x = Math.random()
> >>  ^
> >>
> >> If I rewrite the query slightly, adding in {}, it works:
> >>
> >> val count = spark.parallelize(1 to 1).map(i =>
> >>{
> >>val x = Math.random()
> >>val y = Math.random()
> >>if (x*x + y*y < 1) 1 else 0
> >>}
> >> ).reduce(_ + _)
> >> println("Pi is roughly " + 4.0 * count / 1.0)
> >>
> >>
> >>
> >>
> >>
> >> --
> >> View this message in context:
> >>
> http://apache-spark-developers-list.1001551.n3.nabble.com/Scala-examples-for-Spark-do-not-work-as-written-in-documentation-tp6593.html
> >> Sent from the Apache Spark Developers List mailing list archive at
> >> Nabble.com.
> >>
> >
> >
>


[VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-20 Thread Tathagata Das
Please vote on releasing the following candidate as Apache Spark version 1.0.0!

This has a few bug fixes on top of rc9:
SPARK-1875: https://github.com/apache/spark/pull/824
SPARK-1876: https://github.com/apache/spark/pull/819
SPARK-1878: https://github.com/apache/spark/pull/822
SPARK-1879: https://github.com/apache/spark/pull/823

The tag to be voted on is v1.0.0-rc10 (commit d8070234):
https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=d807023479ce10aec28ef3c1ab646ddefc2e663c

The release files, including signatures, digests, etc. can be found at:
http://people.apache.org/~tdas/spark-1.0.0-rc10/

The release artifacts are signed with the following key:
https://people.apache.org/keys/committer/tdas.asc

The staging repository for this release can be found at:
https://repository.apache.org/content/repositories/orgapachespark-1018/

The documentation corresponding to this release can be found at:
http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/

The full list of changes in this release can be found at:
https://git-wip-us.apache.org/repos/asf?p=spark.git;a=blob;f=CHANGES.txt;h=d21f0ace6326e099360975002797eb7cba9d5273;hb=d807023479ce10aec28ef3c1ab646ddefc2e663c

Please vote on releasing this package as Apache Spark 1.0.0!

The vote is open until Friday, May 23, at 20:00 UTC and passes if
amajority of at least 3 +1 PMC votes are cast.

[ ] +1 Release this package as Apache Spark 1.0.0
[ ] -1 Do not release this package because ...

To learn more about Apache Spark, please see
http://spark.apache.org/

== API Changes ==
We welcome users to compile Spark applications against 1.0. There are
a few API changes in this release. Here are links to the associated
upgrade guides - user facing changes have been kept as small as
possible.

Changes to ML vector specification:
http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/mllib-guide.html#from-09-to-10

Changes to the Java API:
http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/java-programming-guide.html#upgrading-from-pre-10-versions-of-spark

Changes to the streaming API:
http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x

Changes to the GraphX API:
http://people.apache.org/~tdas/spark-1.0.0-rc10-docs/graphx-programming-guide.html#upgrade-guide-from-spark-091

Other changes:
coGroup and related functions now return Iterable[T] instead of Seq[T]
==> Call toSeq on the result to restore the old behavior

SparkContext.jarOfClass returns Option[String] instead of Seq[String]
==> Call toSeq on the result to restore old behavior


Re: BUG: graph.triplets does not return proper values

2014-05-20 Thread GlennStrycker
For some reason it does not appear when I hit "tab" in Spark shell, but when
I put everything together in one line, it DOES WORK!

orig_graph.edges.map(_.copy()).cartesian(orig_graph.edges.map(_.copy())).flatMap(
A => Seq(if (A._1.srcId == A._2.dstId) Edge(A._2.srcId,A._1.dstId,1) else if
(A._1.dstId == A._2.srcId) Edge(A._1.srcId,A._2.dstId,1) else Edge(0,0,0) )
).map(word => (word, 1)).reduceByKey(_ + _).collect

= Array((Edge(5,7,1),4), (Edge(5,6,1),4), (Edge(3,2,1),4), (Edge(5,5,1),3),
(Edge(1,3,1),4), (Edge(2,3,1),4), (Edge(6,5,1),4), (Edge(5,4,1),2),
(Edge(2,1,1),2), (Edge(6,7,1),2), (Edge(2,2,1),2), (Edge(7,5,1),4),
(Edge(3,1,1),4), (Edge(4,5,1),2), (Edge(0,0,0),192), (Edge(3,3,1),3),
(Edge(4,7,1),2), (Edge(1,2,1),2), (Edge(4,4,1),1), (Edge(6,6,1),2),
(Edge(7,4,1),2), (Edge(7,6,1),2), (Edge(7,7,1),2), (Edge(1,1,1),3))




--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/BUG-graph-triplets-does-not-return-proper-values-tp6693p6730.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.


Re: BUG: graph.triplets does not return proper values

2014-05-20 Thread Sean Owen
http://spark.apache.org/docs/0.9.1/api/core/index.html#org.apache.spark.rdd.PairRDDFunctions

It becomes automagically available when your RDD contains pairs.

On Tue, May 20, 2014 at 9:00 PM, GlennStrycker  wrote:
> I don't seem to have this function in my Spark installation for this object,
> or the classes MappedRDD, FlatMappedRDD, EdgeRDD, VertexRDD, or Graph.
>
> Which class should have the reduceByKey function, and how do I cast my
> current RDD as this class?
>
> Perhaps this is still due to my Spark installation being out-of-date?
>
>
>
> --
> View this message in context: 
> http://apache-spark-developers-list.1001551.n3.nabble.com/BUG-graph-triplets-does-not-return-proper-values-tp6693p6728.html
> Sent from the Apache Spark Developers List mailing list archive at Nabble.com.


Re: BUG: graph.triplets does not return proper values

2014-05-20 Thread Mark Hamstra
That's all very old functionality in Spark terms, so it shouldn't have
anything to do with your installation being out-of-date.  There is also no
need to cast as long as the relevant implicit conversions are in scope:
import org.apache.spark.SparkContext._


On Tue, May 20, 2014 at 1:00 PM, GlennStrycker wrote:

> I don't seem to have this function in my Spark installation for this
> object,
> or the classes MappedRDD, FlatMappedRDD, EdgeRDD, VertexRDD, or Graph.
>
> Which class should have the reduceByKey function, and how do I cast my
> current RDD as this class?
>
> Perhaps this is still due to my Spark installation being out-of-date?
>
>
>
> --
> View this message in context:
> http://apache-spark-developers-list.1001551.n3.nabble.com/BUG-graph-triplets-does-not-return-proper-values-tp6693p6728.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>


Re: BUG: graph.triplets does not return proper values

2014-05-20 Thread GlennStrycker
I don't seem to have this function in my Spark installation for this object,
or the classes MappedRDD, FlatMappedRDD, EdgeRDD, VertexRDD, or Graph.

Which class should have the reduceByKey function, and how do I cast my
current RDD as this class?

Perhaps this is still due to my Spark installation being out-of-date?



--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/BUG-graph-triplets-does-not-return-proper-values-tp6693p6728.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.


Re: BUG: graph.triplets does not return proper values

2014-05-20 Thread Reynold Xin
You are probably looking for reduceByKey in that case.

"reduce" just reduces everything in the collection into a single element.


On Tue, May 20, 2014 at 12:16 PM, GlennStrycker wrote:

> Wait a minute... doesn't a reduce function return 1 element PER key pair?
> For example, word-count mapreduce functions return a {word, count} element
> for every unique word.  Is this supposed to be a 1-element RDD object?
>
> The .reduce function for a MappedRDD or FlatMappedRDD both are of the form
>
> def reduce(f: (T, T) => T): T
>
> So presumably if I pass the reduce function a list of values {(X,1), (X,1),
> (X,1), (Y,1), (Y,1)} and the function is ( (A,B) => (A._1, A._2+B._2 ) ),
> then I should get a final vector of {(X,3), (Y,2)}, correct?
>
>
> I have the following object:
>
> scala> temp3
> res128: org.apache.spark.rdd.RDD[(org.apache.spark.graphx.Edge[Int],
> Int)] = MappedRDD[107] at map at :27
>
> and it contains the following:
>
> scala> temp3.collect
> . . .
> res129: Array[(org.apache.spark.graphx.Edge[Int], Int)] =
> Array((Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
> (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
> (Edge(4,4,1),1), (Edge(5,4,1),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
> (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(7,4,1),1), (Edge(0,0,0),1),
> (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
> (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
> (Edge(4,5,1),1), (Edge(5,5,1),1), (Edge(1,2,1),1), (Edge(1,3,1),1),
> (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(7,5,1),1), (Edge(0,0,0),1),
> (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
> (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
> (Edge(4,7,1),1), (Edge(5,7,1),1), (Edge(0,0,0),1), (E...
>
> but when I run the following, I only get one element in the final vector:
>
> scala> temp3.reduce( (A,B) => (A._1, A._2+B._2 ) )
> . . .
> res130: (org.apache.spark.graphx.Edge[Int], Int) = (Edge(0,0,0),256)
>
> I should be additionally getting { (Edge(1,2,1),1), (Edge(1,3,1),2),
> (Edge(2,3,1),2), (Edge(4,5,1),1), (Edge(5,6,1),2), (Edge(6,7,1),1),
> (Edge(4,7,1),1), (Edge(5,7,1),2) }
>
>
>
> Am I not mapping something correctly before running reduce?  I've tried
> both
> .map and .flatMap, and put in _.copy() everywhere, e.g.
>
> temp3.flatMap(A => Seq(A)).reduce( (A,B) => (A._1, A._2+B._2 ) )
> temp3.map(_.copy()).flatMap(A => Seq(A)).reduce( (A,B) => (A._1, A._2+B._2
> )
> )
> etc.
>
>
>
>
>
> --
> View this message in context:
> http://apache-spark-developers-list.1001551.n3.nabble.com/BUG-graph-triplets-does-not-return-proper-values-tp6693p6726.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>


Re: BUG: graph.triplets does not return proper values

2014-05-20 Thread GlennStrycker
Wait a minute... doesn't a reduce function return 1 element PER key pair? 
For example, word-count mapreduce functions return a {word, count} element
for every unique word.  Is this supposed to be a 1-element RDD object?

The .reduce function for a MappedRDD or FlatMappedRDD both are of the form

def reduce(f: (T, T) => T): T

So presumably if I pass the reduce function a list of values {(X,1), (X,1),
(X,1), (Y,1), (Y,1)} and the function is ( (A,B) => (A._1, A._2+B._2 ) ),
then I should get a final vector of {(X,3), (Y,2)}, correct?


I have the following object:

scala> temp3
res128: org.apache.spark.rdd.RDD[(org.apache.spark.graphx.Edge[Int],
Int)] = MappedRDD[107] at map at :27

and it contains the following:

scala> temp3.collect
. . .
res129: Array[(org.apache.spark.graphx.Edge[Int], Int)] =
Array((Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
(Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
(Edge(4,4,1),1), (Edge(5,4,1),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
(Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(7,4,1),1), (Edge(0,0,0),1),
(Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
(Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
(Edge(4,5,1),1), (Edge(5,5,1),1), (Edge(1,2,1),1), (Edge(1,3,1),1),
(Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(7,5,1),1), (Edge(0,0,0),1),
(Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
(Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1), (Edge(0,0,0),1),
(Edge(4,7,1),1), (Edge(5,7,1),1), (Edge(0,0,0),1), (E...

but when I run the following, I only get one element in the final vector:

scala> temp3.reduce( (A,B) => (A._1, A._2+B._2 ) )
. . .
res130: (org.apache.spark.graphx.Edge[Int], Int) = (Edge(0,0,0),256)

I should be additionally getting { (Edge(1,2,1),1), (Edge(1,3,1),2),
(Edge(2,3,1),2), (Edge(4,5,1),1), (Edge(5,6,1),2), (Edge(6,7,1),1),
(Edge(4,7,1),1), (Edge(5,7,1),2) }



Am I not mapping something correctly before running reduce?  I've tried both
.map and .flatMap, and put in _.copy() everywhere, e.g.

temp3.flatMap(A => Seq(A)).reduce( (A,B) => (A._1, A._2+B._2 ) )
temp3.map(_.copy()).flatMap(A => Seq(A)).reduce( (A,B) => (A._1, A._2+B._2 )
)
etc.





--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/BUG-graph-triplets-does-not-return-proper-values-tp6693p6726.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.


Re: Sorting partitions in Java

2014-05-20 Thread Madhu
Sean,

No, I don't want to sort the whole RDD, sortByKey seems to be good enough
for that.

Right now, I think the code I have will work for me, but I can imagine
conditions where it will run out of memory.

I'm not completely sure if  SPARK-983
Andrew mentioned covers
the rdd.sortPartitions() use case. Can someone comment on the scope of
SPARK-983?

Thanks!



-
--
Madhu
https://www.linkedin.com/in/msiddalingaiah
--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/Sorting-partitions-in-Java-tp6715p6725.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.


Re: Sorting partitions in Java

2014-05-20 Thread Sean Owen
On Tue, May 20, 2014 at 6:10 PM, Madhu  wrote:
> What you suggest looks an in-memory sort, which is fine if each partition is
> small enough to fit in memory. Is it true that rdd.sortByKey(...) requires
> partitions to fit in memory? I wasn't sure if there was some magic behind
> the scenes that supports arbitrarily large sorts.

Yes, but so did the Scala version you posted -- I assumed that was OK
for your use case. Regardless of what Spark does, you would copy all
values into memory with toArray.

sortByKey is something fairly different. It sorts the whole RDD by
key, not values within each key. I think Sandy is talking about
something related but not quite the same.

Do you really mean you want to sort the whole RDD?


Re: Sorting partitions in Java

2014-05-20 Thread Andrew Ash
Voted :)

https://issues.apache.org/jira/browse/SPARK-983


On Tue, May 20, 2014 at 10:21 AM, Sandy Ryza wrote:

> There is: SPARK-545
>
>
> On Tue, May 20, 2014 at 10:16 AM, Andrew Ash  wrote:
>
> > Sandy, is there a Jira ticket for that?
> >
> >
> > On Tue, May 20, 2014 at 10:12 AM, Sandy Ryza  > >wrote:
> >
> > > sortByKey currently requires partitions to fit in memory, but there are
> > > plans to add external sort
> > >
> > >
> > > On Tue, May 20, 2014 at 10:10 AM, Madhu  wrote:
> > >
> > > > Thanks Sean, I had seen that post you mentioned.
> > > >
> > > > What you suggest looks an in-memory sort, which is fine if each
> > partition
> > > > is
> > > > small enough to fit in memory. Is it true that rdd.sortByKey(...)
> > > requires
> > > > partitions to fit in memory? I wasn't sure if there was some magic
> > behind
> > > > the scenes that supports arbitrarily large sorts.
> > > >
> > > > None of this is a show stopper, it just might require a little more
> > code
> > > on
> > > > the part of the developer. If there's a requirement for Spark
> > partitions
> > > to
> > > > fit in memory, developers will have to be aware of that and plan
> > > > accordingly. One nice feature of Hadoop MR is the ability to sort
> very
> > > > large
> > > > sets without thinking about data size.
> > > >
> > > > In the case that a developer repartitions an RDD such that some
> > > partitions
> > > > don't fit in memory, sorting those partitions requires more work. For
> > > these
> > > > cases, I think there is value in having a robust partition sorting
> > method
> > > > that deals with it efficiently and reliably.
> > > >
> > > > Is there another solution for sorting arbitrarily large partitions?
> If
> > > not,
> > > > I don't mind developing and contributing a solution.
> > > >
> > > >
> > > >
> > > >
> > > > -
> > > > --
> > > > Madhu
> > > > https://www.linkedin.com/in/msiddalingaiah
> > > > --
> > > > View this message in context:
> > > >
> > >
> >
> http://apache-spark-developers-list.1001551.n3.nabble.com/Sorting-partitions-in-Java-tp6715p6719.html
> > > > Sent from the Apache Spark Developers List mailing list archive at
> > > > Nabble.com.
> > > >
> > >
> >
>


Re: Sorting partitions in Java

2014-05-20 Thread Sandy Ryza
There is: SPARK-545


On Tue, May 20, 2014 at 10:16 AM, Andrew Ash  wrote:

> Sandy, is there a Jira ticket for that?
>
>
> On Tue, May 20, 2014 at 10:12 AM, Sandy Ryza  >wrote:
>
> > sortByKey currently requires partitions to fit in memory, but there are
> > plans to add external sort
> >
> >
> > On Tue, May 20, 2014 at 10:10 AM, Madhu  wrote:
> >
> > > Thanks Sean, I had seen that post you mentioned.
> > >
> > > What you suggest looks an in-memory sort, which is fine if each
> partition
> > > is
> > > small enough to fit in memory. Is it true that rdd.sortByKey(...)
> > requires
> > > partitions to fit in memory? I wasn't sure if there was some magic
> behind
> > > the scenes that supports arbitrarily large sorts.
> > >
> > > None of this is a show stopper, it just might require a little more
> code
> > on
> > > the part of the developer. If there's a requirement for Spark
> partitions
> > to
> > > fit in memory, developers will have to be aware of that and plan
> > > accordingly. One nice feature of Hadoop MR is the ability to sort very
> > > large
> > > sets without thinking about data size.
> > >
> > > In the case that a developer repartitions an RDD such that some
> > partitions
> > > don't fit in memory, sorting those partitions requires more work. For
> > these
> > > cases, I think there is value in having a robust partition sorting
> method
> > > that deals with it efficiently and reliably.
> > >
> > > Is there another solution for sorting arbitrarily large partitions? If
> > not,
> > > I don't mind developing and contributing a solution.
> > >
> > >
> > >
> > >
> > > -
> > > --
> > > Madhu
> > > https://www.linkedin.com/in/msiddalingaiah
> > > --
> > > View this message in context:
> > >
> >
> http://apache-spark-developers-list.1001551.n3.nabble.com/Sorting-partitions-in-Java-tp6715p6719.html
> > > Sent from the Apache Spark Developers List mailing list archive at
> > > Nabble.com.
> > >
> >
>


Re: Sorting partitions in Java

2014-05-20 Thread Andrew Ash
Sandy, is there a Jira ticket for that?


On Tue, May 20, 2014 at 10:12 AM, Sandy Ryza wrote:

> sortByKey currently requires partitions to fit in memory, but there are
> plans to add external sort
>
>
> On Tue, May 20, 2014 at 10:10 AM, Madhu  wrote:
>
> > Thanks Sean, I had seen that post you mentioned.
> >
> > What you suggest looks an in-memory sort, which is fine if each partition
> > is
> > small enough to fit in memory. Is it true that rdd.sortByKey(...)
> requires
> > partitions to fit in memory? I wasn't sure if there was some magic behind
> > the scenes that supports arbitrarily large sorts.
> >
> > None of this is a show stopper, it just might require a little more code
> on
> > the part of the developer. If there's a requirement for Spark partitions
> to
> > fit in memory, developers will have to be aware of that and plan
> > accordingly. One nice feature of Hadoop MR is the ability to sort very
> > large
> > sets without thinking about data size.
> >
> > In the case that a developer repartitions an RDD such that some
> partitions
> > don't fit in memory, sorting those partitions requires more work. For
> these
> > cases, I think there is value in having a robust partition sorting method
> > that deals with it efficiently and reliably.
> >
> > Is there another solution for sorting arbitrarily large partitions? If
> not,
> > I don't mind developing and contributing a solution.
> >
> >
> >
> >
> > -
> > --
> > Madhu
> > https://www.linkedin.com/in/msiddalingaiah
> > --
> > View this message in context:
> >
> http://apache-spark-developers-list.1001551.n3.nabble.com/Sorting-partitions-in-Java-tp6715p6719.html
> > Sent from the Apache Spark Developers List mailing list archive at
> > Nabble.com.
> >
>


Re: Sorting partitions in Java

2014-05-20 Thread Sandy Ryza
sortByKey currently requires partitions to fit in memory, but there are
plans to add external sort


On Tue, May 20, 2014 at 10:10 AM, Madhu  wrote:

> Thanks Sean, I had seen that post you mentioned.
>
> What you suggest looks an in-memory sort, which is fine if each partition
> is
> small enough to fit in memory. Is it true that rdd.sortByKey(...) requires
> partitions to fit in memory? I wasn't sure if there was some magic behind
> the scenes that supports arbitrarily large sorts.
>
> None of this is a show stopper, it just might require a little more code on
> the part of the developer. If there's a requirement for Spark partitions to
> fit in memory, developers will have to be aware of that and plan
> accordingly. One nice feature of Hadoop MR is the ability to sort very
> large
> sets without thinking about data size.
>
> In the case that a developer repartitions an RDD such that some partitions
> don't fit in memory, sorting those partitions requires more work. For these
> cases, I think there is value in having a robust partition sorting method
> that deals with it efficiently and reliably.
>
> Is there another solution for sorting arbitrarily large partitions? If not,
> I don't mind developing and contributing a solution.
>
>
>
>
> -
> --
> Madhu
> https://www.linkedin.com/in/msiddalingaiah
> --
> View this message in context:
> http://apache-spark-developers-list.1001551.n3.nabble.com/Sorting-partitions-in-Java-tp6715p6719.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>


Re: Sorting partitions in Java

2014-05-20 Thread Madhu
Thanks Sean, I had seen that post you mentioned.

What you suggest looks an in-memory sort, which is fine if each partition is
small enough to fit in memory. Is it true that rdd.sortByKey(...) requires
partitions to fit in memory? I wasn't sure if there was some magic behind
the scenes that supports arbitrarily large sorts.

None of this is a show stopper, it just might require a little more code on
the part of the developer. If there's a requirement for Spark partitions to
fit in memory, developers will have to be aware of that and plan
accordingly. One nice feature of Hadoop MR is the ability to sort very large
sets without thinking about data size.

In the case that a developer repartitions an RDD such that some partitions
don't fit in memory, sorting those partitions requires more work. For these
cases, I think there is value in having a robust partition sorting method
that deals with it efficiently and reliably.

Is there another solution for sorting arbitrarily large partitions? If not,
I don't mind developing and contributing a solution.




-
--
Madhu
https://www.linkedin.com/in/msiddalingaiah
--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/Sorting-partitions-in-Java-tp6715p6719.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.


Re: Sorting partitions in Java

2014-05-20 Thread Sean Owen
It's an Iterator in both Java and Scala. In both cases you need to
copy the stream of values into something List-like to sort it. An
Iterable would not change that (not sure the API can promise many
iterations anyway).

If you just want the equivalent of "toArray", you can use a utility
method in Commons Collections or Guava. Guava's
Lists.newArrayList(Iterator) does nicely, which you can then
Collections.sort() with a Comparator and the return its iterator()

I dug this up too, remembering a similar question:
http://mail-archives.apache.org/mod_mbox/spark-user/201312.mbox/%3c529f819f.3060...@vu.nl%3E

On Tue, May 20, 2014 at 2:25 PM, Madhu  wrote:
> I'm trying to sort data in each partition of an RDD.
> I was able to do it successfully in Scala like this:
>
> val sorted = rdd.mapPartitions(iter => {
>   iter.toArray.sortWith((x, y) => x._2.compare(y._2) < 0).iterator
> },
> preservesPartitioning = true)
>
> I used the same technique as in OrderedRDDFunctions.scala, so I assume it's
> a reasonable way to do it.
>
> This works well so far, but I can't seem to do the same thing in Java
> because 'iter' in the Java APIs is an Iterator rather than an Iterable.
> There may be an unattractive workaround, but I didn't pursue it.
>
> Ideally, it would be nice to have an efficient, robust method in RDD to sort
> each partition.
> Does something like that exist?
>
> Thanks!
>
>
>
> -
> --
> Madhu
> https://www.linkedin.com/in/msiddalingaiah
> --
> View this message in context: 
> http://apache-spark-developers-list.1001551.n3.nabble.com/Sorting-partitions-in-Java-tp6715.html
> Sent from the Apache Spark Developers List mailing list archive at Nabble.com.


Re: BUG: graph.triplets does not return proper values

2014-05-20 Thread GlennStrycker
Oh... ha, good point.  Sorry, I'm new to mapreduce programming and forgot
about that... I'll have to adjust my reduce function to output a vector/RDD
as the element to return.  Thanks for reminding me of this!



--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/BUG-graph-triplets-does-not-return-proper-values-tp6693p6717.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.


Re: [VOTE] Release Apache Spark 1.0.0 (rc9)

2014-05-20 Thread Tom Graves
I assume we will have an rc10 to fix the issues Matei found?

Tom


On Sunday, May 18, 2014 9:08 PM, Patrick Wendell  wrote:
 


Hey Matei - the issue you found is not related to security. This patch
a few days ago broke builds for Hadoop 1 with YARN support enabled.
The patch directly altered the way we deal with commons-lang
dependency, which is what is at the base of this stack trace.

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

- Patrick


On Sun, May 18, 2014 at 5:28 PM, Matei Zaharia  wrote:
> Alright, I've opened https://github.com/apache/spark/pull/819 with the 
> Windows fixes. I also found one other likely bug, 
> https://issues.apache.org/jira/browse/SPARK-1875, in the binary packages for 
> Hadoop1 built in this RC. I think this is due to Hadoop 1's security code 
> depending on a different version of org.apache.commons than Hadoop 2, but it 
> needs investigation. Tom, any thoughts on this?
>
> Matei
>
> On May 18, 2014, at 12:33 PM, Matei Zaharia  wrote:
>
>> I took the always fun task of testing it on Windows, and unfortunately, I 
>> found some small problems with the prebuilt packages due to recent changes 
>> to the launch scripts: bin/spark-class2.cmd looks in ./jars instead of ./lib 
>> for the assembly JAR, and bin/run-example2.cmd doesn't quite match the 
>> master-setting behavior of the Unix based one. I'll send a pull request to 
>> fix them soon.
>>
>> Matei
>>
>>
>> On May 17, 2014, at 11:32 AM, Sandy Ryza  wrote:
>>
>>> +1
>>>
>>> Reran my tests from rc5:
>>>
>>> * Built the release from source.
>>> * Compiled Java and Scala apps that interact with HDFS against it.
>>> * Ran them in local mode.
>>> * Ran them against a pseudo-distributed YARN cluster in both yarn-client
>>> mode and yarn-cluster mode.
>>>
>>>
>>> On Sat, May 17, 2014 at 10:08 AM, Andrew Or  wrote:
>>>
 +1


 2014-05-17 8:53 GMT-07:00 Mark Hamstra :

> +1
>
>
> On Sat, May 17, 2014 at 12:58 AM, Patrick Wendell > wrote:
>
>> I'll start the voting with a +1.
>>
>> On Sat, May 17, 2014 at 12:58 AM, Patrick Wendell 
>> wrote:
>>> Please vote on releasing the following candidate as Apache Spark
> version
>> 1.0.0!
>>> This has one bug fix and one minor feature on top of rc8:
>>> SPARK-1864: https://github.com/apache/spark/pull/808
>>> SPARK-1808: https://github.com/apache/spark/pull/799
>>>
>>> The tag to be voted on is v1.0.0-rc9 (commit 920f947):
>>>
>>
>
 https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=920f947eb5a22a679c0c3186cf69ee75f6041c75
>>>
>>> The release files, including signatures, digests, etc. can be found
 at:
>>> http://people.apache.org/~pwendell/spark-1.0.0-rc9/
>>>
>>> Release artifacts are signed with the following key:
>>> https://people.apache.org/keys/committer/pwendell.asc
>>>
>>> The staging repository for this release can be found at:
>>>
> https://repository.apache.org/content/repositories/orgapachespark-1017/
>>>
>>> The documentation corresponding to this release can be found at:
>>> http://people.apache.org/~pwendell/spark-1.0.0-rc9-docs/
>>>
>>> Please vote on releasing this package as Apache Spark 1.0.0!
>>>
>>> The vote is open until Tuesday, May 20, at 08:56 UTC and passes if
>>> amajority of at least 3 +1 PMC votes are cast.
>>>
>>> [ ] +1 Release this package as Apache Spark 1.0.0
>>> [ ] -1 Do not release this package because ...
>>>
>>> To learn more about Apache Spark, please see
>>> http://spark.apache.org/
>>>
>>> == API Changes ==
>>> We welcome users to compile Spark applications against 1.0. There are
>>> a few API changes in this release. Here are links to the associated
>>> upgrade guides - user facing changes have been kept as small as
>>> possible.
>>>
>>> changes to ML vector specification:
>>>
>>
>
 http://people.apache.org/~pwendell/spark-1.0.0-rc8-docs/mllib-guide.html#from-09-to-10
>>>
>>> changes to the Java API:
>>>
>>
>
 http://people.apache.org/~pwendell/spark-1.0.0-rc8-docs/java-programming-guide.html#upgrading-from-pre-10-versions-of-spark
>>>
>>> changes to the streaming API:
>>>
>>
>
 http://people.apache.org/~pwendell/spark-1.0.0-rc8-docs/streaming-programming-guide.html#migration-guide-from-091-or-below-to-1x
>>>
>>> changes to the GraphX API:
>>>
>>
>
 http://people.apache.org/~pwendell/spark-1.0.0-rc8-docs/graphx-programming-guide.html#upgrade-guide-from-spark-091
>>>
>>> coGroup and related functions now return Iterable[T] instead of
 Seq[T]
>>> ==> Call toSeq on the result to restore the old behavior
>>>
>>> SparkContext.jarOfClass returns Option[String] instead of Seq[String]
>>> ==> Call toSeq on the result to restore old behavior
>>
>

>>
>

Sorting partitions in Java

2014-05-20 Thread Madhu
I'm trying to sort data in each partition of an RDD.
I was able to do it successfully in Scala like this:

val sorted = rdd.mapPartitions(iter => {
  iter.toArray.sortWith((x, y) => x._2.compare(y._2) < 0).iterator
},
preservesPartitioning = true)

I used the same technique as in OrderedRDDFunctions.scala, so I assume it's
a reasonable way to do it.

This works well so far, but I can't seem to do the same thing in Java
because 'iter' in the Java APIs is an Iterator rather than an Iterable.
There may be an unattractive workaround, but I didn't pursue it.

Ideally, it would be nice to have an efficient, robust method in RDD to sort
each partition.
Does something like that exist?

Thanks!



-
--
Madhu
https://www.linkedin.com/in/msiddalingaiah
--
View this message in context: 
http://apache-spark-developers-list.1001551.n3.nabble.com/Sorting-partitions-in-Java-tp6715.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.