Re: Spark 1.5.0-SNAPSHOT broken with Scala 2.11
On Mon, Jun 29, 2015 at 3:02 AM, Alessandro Baretta alexbare...@gmail.com wrote: I am building the current master branch with Scala 2.11 following these instructions: Building for Scala 2.11 To produce a Spark package compiled with Scala 2.11, use the -Dscala-2.11 property: dev/change-version-to-2.11.sh mvn -Pyarn -Phadoop-2.4 -Dscala-2.11 -DskipTests clean package Here's what I'm seeing: log4j:WARN No appenders could be found for logger (org.apache.hadoop.security.Groups). log4j:WARN Please initialize the log4j system properly. log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more info. Using Spark's repl log4j profile: org/apache/spark/log4j-defaults-repl.properties To adjust logging level use sc.setLogLevel(INFO) Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 1.5.0-SNAPSHOT /_/ Using *Scala version 2.10.4* (OpenJDK 64-Bit Server VM, Java 1.7.0_79) Type in expressions to have them evaluated. Something is deeply wrong with your build. iulian Type :help for more information. 15/06/29 00:42:20 ERROR ActorSystemImpl: Uncaught fatal error from thread [sparkDriver-akka.remote.default-remote-dispatcher-6] shutting down ActorSystem [sparkDriver] java.lang.VerifyError: class akka.remote.WireFormats$AkkaControlMessage overrides final method getUnknownFields.()Lcom/google/protobuf/UnknownFieldSet; at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:800) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:449) at java.net.URLClassLoader.access$100(URLClassLoader.java:71) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) at akka.remote.transport.AkkaPduProtobufCodec$.constructControlMessagePdu(AkkaPduCodec.scala:231) at akka.remote.transport.AkkaPduProtobufCodec$.init(AkkaPduCodec.scala:153) at akka.remote.transport.AkkaPduProtobufCodec$.clinit(AkkaPduCodec.scala) at akka.remote.EndpointManager$$anonfun$9.apply(Remoting.scala:733) at akka.remote.EndpointManager$$anonfun$9.apply(Remoting.scala:703) What am I doing wrong? -- -- Iulian Dragos -- Reactive Apps on the JVM www.typesafe.com
Re: [VOTE] Release Apache Spark 1.4.1
+1 sigs, license, etc check out. All tests pass for me in the Hadoop 2.6 + Hive configuration on Ubuntu. (I still get those pesky cosmetic UDF test failures in Java 8, but they are clearly just test issues.) I'll follow up on retargeting 1.4.1 issues afterwards as needed, but again feel free to move those you're sure won't be in this release. On Wed, Jun 24, 2015 at 6:37 AM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: [VOTE] Release Apache Spark 1.4.1
+1. Tested on yarn on hadoop 2.6 cluster Tom On Monday, June 29, 2015 2:04 AM, Tathagata Das tathagata.das1...@gmail.com wrote: @Ted, could you elaborate more on what was the test command that you ran? What profiles, using SBT or Maven? TD On Sun, Jun 28, 2015 at 12:21 PM, Patrick Wendell pwend...@gmail.com wrote: Hey Krishna - this is still the current release candidate. - Patrick On Sun, Jun 28, 2015 at 12:14 PM, Krishna Sankar ksanka...@gmail.com wrote: Patrick, Haven't seen any replies on test results. I will byte ;o) - Should I test this version or is another one in the wings ? Cheers k/ On Tue, Jun 23, 2015 at 10:37 PM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: how can I write a language wrapper?
Hi Vasili, It so happens that the entire SparkR code was merged to Apache Spark in a single pull request. So you can see at once all the required changes in https://github.com/apache/spark/pull/5096. It's 12,043 lines and took more than 20 people about a year to write as I understand it. On Mon, Jun 29, 2015 at 10:33 AM, Vasili I. Galchin vigalc...@gmail.com wrote: Shivaram, Vis-a-vis Haskell support, I am reading DataFrame.R, SparkRBackend*, context.R, et. al., am I headed in the correct direction?/ Yes or no, please give more guidance. Thank you. Kind regards, Vasili On Tue, Jun 23, 2015 at 1:46 PM, Shivaram Venkataraman shiva...@eecs.berkeley.edu wrote: Every language has its own quirks / features -- so I don't think there exists a document on how to go about doing this for a new language. The most related write up I know of is the wiki page on PySpark internals https://cwiki.apache.org/confluence/display/SPARK/PySpark+Internals written by Josh Rosen -- It covers some of the issues like closure capture, serialization, JVM communication that you'll need to handle for a new language. Thanks Shivaram On Tue, Jun 23, 2015 at 1:35 PM, Vasili I. Galchin vigalc...@gmail.com wrote: Hello, I want to add language support for another language(other than Scala, Java et. al.). Where is documentation that explains to provide support for a new language? Thank you, Vasili - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: Spark 1.5.0-SNAPSHOT broken with Scala 2.11
On 29 Jun 2015, at 11:27, Iulian Dragoș iulian.dra...@typesafe.commailto:iulian.dra...@typesafe.com wrote: On Mon, Jun 29, 2015 at 3:02 AM, Alessandro Baretta alexbare...@gmail.commailto:alexbare...@gmail.com wrote: I am building the current master branch with Scala 2.11 following these instructions: Type :help for more information. 15/06/29 00:42:20 ERROR ActorSystemImpl: Uncaught fatal error from thread [sparkDriver-akka.remote.default-remote-dispatcher-6] shutting down ActorSystem [sparkDriver] java.lang.VerifyError: class akka.remote.WireFormats$AkkaControlMessage overrides final method getUnknownFields.()Lcom/google/protobuf/UnknownFieldSet; at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:800) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:449) at java.net.URLClassLoader.access$100(URLClassLoader.java:71) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) at akka.remote.transport.AkkaPduProtobufCodec$.constructControlMessagePdu(AkkaPduCodec.scala:231) at akka.remote.transport.AkkaPduProtobufCodec$.init(AkkaPduCodec.scala:153) at akka.remote.transport.AkkaPduProtobufCodec$.clinit(AkkaPduCodec.scala) at akka.remote.EndpointManager$$anonfun$9.apply(Remoting.scala:733) at akka.remote.EndpointManager$$anonfun$9.apply(Remoting.scala:703) What am I doing wrong? oh, that's just the version of the protoc protobuf compiler generating code that implementation classes aren't compatible with, and/or the version of probuf.jar on the classpath. Google's libraries are turning out to be surprisingly brittle that way. when you type protoc --version on the command line, you should expect to see, libprotoc 2.5.0'; and have protobuf-2.5.0 on the classpath. If neither of those conditions are met: fix them -Steve
Re: [VOTE] Release Apache Spark 1.4.1
+1. I tested those SQL blocker bugs in my laptop and they have been fixed. On Mon, Jun 29, 2015 at 6:51 AM, Sean Owen so...@cloudera.com wrote: +1 sigs, license, etc check out. All tests pass for me in the Hadoop 2.6 + Hive configuration on Ubuntu. (I still get those pesky cosmetic UDF test failures in Java 8, but they are clearly just test issues.) I'll follow up on retargeting 1.4.1 issues afterwards as needed, but again feel free to move those you're sure won't be in this release. On Wed, Jun 24, 2015 at 6:37 AM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: [VOTE] Release Apache Spark 1.4.1
Here is the command I used: mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive package Java: 1.8.0_45 OS: Linux x.com 2.6.32-504.el6.x86_64 #1 SMP Wed Oct 15 04:27:16 UTC 2014 x86_64 x86_64 x86_64 GNU/Linux Cheers On Mon, Jun 29, 2015 at 12:04 AM, Tathagata Das tathagata.das1...@gmail.com wrote: @Ted, could you elaborate more on what was the test command that you ran? What profiles, using SBT or Maven? TD On Sun, Jun 28, 2015 at 12:21 PM, Patrick Wendell pwend...@gmail.com wrote: Hey Krishna - this is still the current release candidate. - Patrick On Sun, Jun 28, 2015 at 12:14 PM, Krishna Sankar ksanka...@gmail.com wrote: Patrick, Haven't seen any replies on test results. I will byte ;o) - Should I test this version or is another one in the wings ? Cheers k/ On Tue, Jun 23, 2015 at 10:37 PM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: Spark 1.5.0-SNAPSHOT broken with Scala 2.11
Steve, It was indeed a protocol buffers issue. I am able to build spark now. Thanks. On Mon, Jun 29, 2015 at 7:37 AM, Steve Loughran ste...@hortonworks.com wrote: On 29 Jun 2015, at 11:27, Iulian Dragoș iulian.dra...@typesafe.com wrote: On Mon, Jun 29, 2015 at 3:02 AM, Alessandro Baretta alexbare...@gmail.com wrote: I am building the current master branch with Scala 2.11 following these instructions: Type :help for more information. 15/06/29 00:42:20 ERROR ActorSystemImpl: Uncaught fatal error from thread [sparkDriver-akka.remote.default-remote-dispatcher-6] shutting down ActorSystem [sparkDriver] java.lang.VerifyError: class akka.remote.WireFormats$AkkaControlMessage overrides final method getUnknownFields.()Lcom/google/protobuf/UnknownFieldSet; at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:800) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:449) at java.net.URLClassLoader.access$100(URLClassLoader.java:71) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) at akka.remote.transport.AkkaPduProtobufCodec$.constructControlMessagePdu(AkkaPduCodec.scala:231) at akka.remote.transport.AkkaPduProtobufCodec$.init(AkkaPduCodec.scala:153) at akka.remote.transport.AkkaPduProtobufCodec$.clinit(AkkaPduCodec.scala) at akka.remote.EndpointManager$$anonfun$9.apply(Remoting.scala:733) at akka.remote.EndpointManager$$anonfun$9.apply(Remoting.scala:703) What am I doing wrong? oh, that's just the version of the protoc protobuf compiler generating code that implementation classes aren't compatible with, and/or the version of probuf.jar on the classpath. Google's libraries are turning out to be surprisingly brittle that way. when you type protoc --version on the command line, you should expect to see, libprotoc 2.5.0'; and have protobuf-2.5.0 on the classpath. If neither of those conditions are met: fix them -Steve
Re: [VOTE] Release Apache Spark 1.4.1
The test passes when run alone on my machine as well. Please run test suite. Thanks On Mon, Jun 29, 2015 at 2:01 PM, Tathagata Das tathagata.das1...@gmail.com wrote: @Ted, I ran the following two commands. mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive -DskipTests clean package mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive -DwildcardSuites=org.apache.spark.streaming.StreamingContextSuite test Using Java version 1.7.0_51, the tests passed normally. On Mon, Jun 29, 2015 at 1:05 PM, Krishna Sankar ksanka...@gmail.com wrote: +1 (non-binding, of course) 1. Compiled OSX 10.10 (Yosemite) OK Total time: 13:26 min mvn clean package -Pyarn -Phadoop-2.6 -DskipTests 2. Tested pyspark, mllib 2.1. statistics (min,max,mean,Pearson,Spearman) OK 2.2. Linear/Ridge/Laso Regression OK 2.3. Decision Tree, Naive Bayes OK 2.4. KMeans OK Center And Scale OK 2.5. RDD operations OK State of the Union Texts - MapReduce, Filter,sortByKey (word count) 2.6. Recommendation (Movielens medium dataset ~1 M ratings) OK Model evaluation/optimization (rank, numIter, lambda) with itertools OK 3. Scala - MLlib 3.1. statistics (min,max,mean,Pearson,Spearman) OK 3.2. LinearRegressionWithSGD OK 3.3. Decision Tree OK 3.4. KMeans OK 3.5. Recommendation (Movielens medium dataset ~1 M ratings) OK 3.6. saveAsParquetFile OK 3.7. Read and verify the 4.3 save(above) - sqlContext.parquetFile, registerTempTable, sql OK 3.8. result = sqlContext.sql(SELECT OrderDetails.OrderID,ShipCountry,UnitPrice,Qty,Discount FROM Orders INNER JOIN OrderDetails ON Orders.OrderID = OrderDetails.OrderID) OK 4.0. Spark SQL from Python OK 4.1. result = sqlContext.sql(SELECT * from people WHERE State = 'WA') OK 5.0. Packages 5.1. com.databricks.spark.csv - read/write OK Cheers k/ On Tue, Jun 23, 2015 at 10:37 PM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: [VOTE] Release Apache Spark 1.4.1
Hi Ted, We haven't observed a StreamingContextSuite failure on our test infrastructure recently. Given that we cannot reproduce it even locally it is unlikely that this uncovers a real bug. Even if it does I would not block the release on it because many in the community are waiting for a few important fixes. In general, there will always be outstanding issues in Spark that we cannot address in every release. -Andrew 2015-06-29 14:29 GMT-07:00 Ted Yu yuzhih...@gmail.com: The test passes when run alone on my machine as well. Please run test suite. Thanks On Mon, Jun 29, 2015 at 2:01 PM, Tathagata Das tathagata.das1...@gmail.com wrote: @Ted, I ran the following two commands. mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive -DskipTests clean package mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive -DwildcardSuites=org.apache.spark.streaming.StreamingContextSuite test Using Java version 1.7.0_51, the tests passed normally. On Mon, Jun 29, 2015 at 1:05 PM, Krishna Sankar ksanka...@gmail.com wrote: +1 (non-binding, of course) 1. Compiled OSX 10.10 (Yosemite) OK Total time: 13:26 min mvn clean package -Pyarn -Phadoop-2.6 -DskipTests 2. Tested pyspark, mllib 2.1. statistics (min,max,mean,Pearson,Spearman) OK 2.2. Linear/Ridge/Laso Regression OK 2.3. Decision Tree, Naive Bayes OK 2.4. KMeans OK Center And Scale OK 2.5. RDD operations OK State of the Union Texts - MapReduce, Filter,sortByKey (word count) 2.6. Recommendation (Movielens medium dataset ~1 M ratings) OK Model evaluation/optimization (rank, numIter, lambda) with itertools OK 3. Scala - MLlib 3.1. statistics (min,max,mean,Pearson,Spearman) OK 3.2. LinearRegressionWithSGD OK 3.3. Decision Tree OK 3.4. KMeans OK 3.5. Recommendation (Movielens medium dataset ~1 M ratings) OK 3.6. saveAsParquetFile OK 3.7. Read and verify the 4.3 save(above) - sqlContext.parquetFile, registerTempTable, sql OK 3.8. result = sqlContext.sql(SELECT OrderDetails.OrderID,ShipCountry,UnitPrice,Qty,Discount FROM Orders INNER JOIN OrderDetails ON Orders.OrderID = OrderDetails.OrderID) OK 4.0. Spark SQL from Python OK 4.1. result = sqlContext.sql(SELECT * from people WHERE State = 'WA') OK 5.0. Packages 5.1. com.databricks.spark.csv - read/write OK Cheers k/ On Tue, Jun 23, 2015 at 10:37 PM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: [VOTE] Release Apache Spark 1.4.1
+1 (non-binding, of course) 1. Compiled OSX 10.10 (Yosemite) OK Total time: 13:26 min mvn clean package -Pyarn -Phadoop-2.6 -DskipTests 2. Tested pyspark, mllib 2.1. statistics (min,max,mean,Pearson,Spearman) OK 2.2. Linear/Ridge/Laso Regression OK 2.3. Decision Tree, Naive Bayes OK 2.4. KMeans OK Center And Scale OK 2.5. RDD operations OK State of the Union Texts - MapReduce, Filter,sortByKey (word count) 2.6. Recommendation (Movielens medium dataset ~1 M ratings) OK Model evaluation/optimization (rank, numIter, lambda) with itertools OK 3. Scala - MLlib 3.1. statistics (min,max,mean,Pearson,Spearman) OK 3.2. LinearRegressionWithSGD OK 3.3. Decision Tree OK 3.4. KMeans OK 3.5. Recommendation (Movielens medium dataset ~1 M ratings) OK 3.6. saveAsParquetFile OK 3.7. Read and verify the 4.3 save(above) - sqlContext.parquetFile, registerTempTable, sql OK 3.8. result = sqlContext.sql(SELECT OrderDetails.OrderID,ShipCountry,UnitPrice,Qty,Discount FROM Orders INNER JOIN OrderDetails ON Orders.OrderID = OrderDetails.OrderID) OK 4.0. Spark SQL from Python OK 4.1. result = sqlContext.sql(SELECT * from people WHERE State = 'WA') OK 5.0. Packages 5.1. com.databricks.spark.csv - read/write OK Cheers k/ On Tue, Jun 23, 2015 at 10:37 PM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: [VOTE] Release Apache Spark 1.4.1
Andrew: I agree with your assessment. Cheers On Mon, Jun 29, 2015 at 3:33 PM, Andrew Or and...@databricks.com wrote: Hi Ted, We haven't observed a StreamingContextSuite failure on our test infrastructure recently. Given that we cannot reproduce it even locally it is unlikely that this uncovers a real bug. Even if it does I would not block the release on it because many in the community are waiting for a few important fixes. In general, there will always be outstanding issues in Spark that we cannot address in every release. -Andrew 2015-06-29 14:29 GMT-07:00 Ted Yu yuzhih...@gmail.com: The test passes when run alone on my machine as well. Please run test suite. Thanks On Mon, Jun 29, 2015 at 2:01 PM, Tathagata Das tathagata.das1...@gmail.com wrote: @Ted, I ran the following two commands. mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive -DskipTests clean package mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive -DwildcardSuites=org.apache.spark.streaming.StreamingContextSuite test Using Java version 1.7.0_51, the tests passed normally. On Mon, Jun 29, 2015 at 1:05 PM, Krishna Sankar ksanka...@gmail.com wrote: +1 (non-binding, of course) 1. Compiled OSX 10.10 (Yosemite) OK Total time: 13:26 min mvn clean package -Pyarn -Phadoop-2.6 -DskipTests 2. Tested pyspark, mllib 2.1. statistics (min,max,mean,Pearson,Spearman) OK 2.2. Linear/Ridge/Laso Regression OK 2.3. Decision Tree, Naive Bayes OK 2.4. KMeans OK Center And Scale OK 2.5. RDD operations OK State of the Union Texts - MapReduce, Filter,sortByKey (word count) 2.6. Recommendation (Movielens medium dataset ~1 M ratings) OK Model evaluation/optimization (rank, numIter, lambda) with itertools OK 3. Scala - MLlib 3.1. statistics (min,max,mean,Pearson,Spearman) OK 3.2. LinearRegressionWithSGD OK 3.3. Decision Tree OK 3.4. KMeans OK 3.5. Recommendation (Movielens medium dataset ~1 M ratings) OK 3.6. saveAsParquetFile OK 3.7. Read and verify the 4.3 save(above) - sqlContext.parquetFile, registerTempTable, sql OK 3.8. result = sqlContext.sql(SELECT OrderDetails.OrderID,ShipCountry,UnitPrice,Qty,Discount FROM Orders INNER JOIN OrderDetails ON Orders.OrderID = OrderDetails.OrderID) OK 4.0. Spark SQL from Python OK 4.1. result = sqlContext.sql(SELECT * from people WHERE State = 'WA') OK 5.0. Packages 5.1. com.databricks.spark.csv - read/write OK Cheers k/ On Tue, Jun 23, 2015 at 10:37 PM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: Question about Spark process and thread
Most of those threads are not for task execution. They are for RPC, scheduling, ... On Sun, Jun 28, 2015 at 8:32 AM, Dogtail Ray spark.ru...@gmail.com wrote: Hi, I was looking at Spark source code, and I found that when launching a Executor, actually Spark is launching a threadpool; each time the scheduler launches a task, the executor will launch a thread within the threadpool. However, I also found that the Spark process always has approximately 40 threads running regardless of my configuration (SPARK_WORKER_CORES, SPARK_WORKER_INSTANCES, --executor-cores, --total-executor-cores, etc.). Does it mean Spark will pre-launch 40 threads even before the tasks are launched? Great thanks! Best, Ray
Dataframes filter by count fails with python API
Dear developers, I found the following behaviour that I think is a minor bug. If I apply groupBy and count in python API, the resulting data frame has grouped columns and the field named count. Filtering by that field does not work because it thinks it is a key word: x = sc.parallelize(zip(xrange(1000),xrange(1000))) df = sqlContext.createDataFrame(x) df.groupBy(_1).count().printSchema() root |-- _1: long (nullable = true) |-- count: long (nullable = false) df.groupBy(_1).count().filter(count 1) gives : java.lang.RuntimeException: [1.7] failure: ``('' expected but `' found count 1 ^ at scala.sys.package$.error(package.scala:27) the following syntax works : f = df.groupBy(_1).count() n = f.filter(f[count] 1) In Scala, referring to $count column works as well. please let me know if I should submit a JIRA for this.
Re: how can I write a language wrapper?
Shivaram, Vis-a-vis Haskell support, I am reading DataFrame.R, SparkRBackend*, context.R, et. al., am I headed in the correct direction?/ Yes or no, please give more guidance. Thank you. Kind regards, Vasili On Tue, Jun 23, 2015 at 1:46 PM, Shivaram Venkataraman shiva...@eecs.berkeley.edu wrote: Every language has its own quirks / features -- so I don't think there exists a document on how to go about doing this for a new language. The most related write up I know of is the wiki page on PySpark internals https://cwiki.apache.org/confluence/display/SPARK/PySpark+Internals written by Josh Rosen -- It covers some of the issues like closure capture, serialization, JVM communication that you'll need to handle for a new language. Thanks Shivaram On Tue, Jun 23, 2015 at 1:35 PM, Vasili I. Galchin vigalc...@gmail.com wrote: Hello, I want to add language support for another language(other than Scala, Java et. al.). Where is documentation that explains to provide support for a new language? Thank you, Vasili - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: Dataframes filter by count fails with python API
Hi Andrew, Thanks for the email. This is a known bug with the expression parser. We will hopefully fix this in 1.5. There are more keywords with the expression parser, and we have already got rid of most of them. Count is still there due to the handling of count distinct, but we plan to get rid of that as well. On Sun, Jun 28, 2015 at 11:57 PM, Andrew Vykhodtsev yoz...@gmail.com wrote: Dear developers, I found the following behaviour that I think is a minor bug. If I apply groupBy and count in python API, the resulting data frame has grouped columns and the field named count. Filtering by that field does not work because it thinks it is a key word: x = sc.parallelize(zip(xrange(1000),xrange(1000))) df = sqlContext.createDataFrame(x) df.groupBy(_1).count().printSchema() root |-- _1: long (nullable = true) |-- count: long (nullable = false) df.groupBy(_1).count().filter(count 1) gives : java.lang.RuntimeException: [1.7] failure: ``('' expected but `' found count 1 ^ at scala.sys.package$.error(package.scala:27) the following syntax works : f = df.groupBy(_1).count() n = f.filter(f[count] 1) In Scala, referring to $count column works as well. please let me know if I should submit a JIRA for this.
Re: [VOTE] Release Apache Spark 1.4.1
@Ted, could you elaborate more on what was the test command that you ran? What profiles, using SBT or Maven? TD On Sun, Jun 28, 2015 at 12:21 PM, Patrick Wendell pwend...@gmail.com wrote: Hey Krishna - this is still the current release candidate. - Patrick On Sun, Jun 28, 2015 at 12:14 PM, Krishna Sankar ksanka...@gmail.com wrote: Patrick, Haven't seen any replies on test results. I will byte ;o) - Should I test this version or is another one in the wings ? Cheers k/ On Tue, Jun 23, 2015 at 10:37 PM, Patrick Wendell pwend...@gmail.com wrote: Please vote on releasing the following candidate as Apache Spark version 1.4.1! This release fixes a handful of known issues in Spark 1.4.0, listed here: http://s.apache.org/spark-1.4.1 The tag to be voted on is v1.4.1-rc1 (commit 60e08e5): https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h= 60e08e50751fe3929156de956d62faea79f5b801 The release files, including signatures, digests, etc. can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-bin/ 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: [published as version: 1.4.1] https://repository.apache.org/content/repositories/orgapachespark-1118/ [published as version: 1.4.1-rc1] https://repository.apache.org/content/repositories/orgapachespark-1119/ The documentation corresponding to this release can be found at: http://people.apache.org/~pwendell/spark-releases/spark-1.4.1-rc1-docs/ Please vote on releasing this package as Apache Spark 1.4.1! The vote is open until Saturday, June 27, at 06:32 UTC and passes if a majority of at least 3 +1 PMC votes are cast. [ ] +1 Release this package as Apache Spark 1.4.1 [ ] -1 Do not release this package because ... To learn more about Apache Spark, please see http://spark.apache.org/ - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org
Re: how can I write a language wrapper?
My guess is that if you are just wrapping the spark sql APIs, you can get away with not having to reimplement a lot of the complexities in Pyspark like storing everything in RDDs as pickled byte arrays, pipelining RDDs, doing aggregations and joins in the python interpreters, etc. Since the canonical representation of objects in Spark SQL is in scala/jvm, you're effectively just proxying calls to the java side. The only tricky thing is UDFs, which naturally need to run in an interpreter of the wrapper language. I'm currently thinking of redesigning the UDFs to be sent in a language agnostic data format like protobufs or msgpack, so that all language wrappers just need to implement the simple protocol of reading those in, transforming it, then outputting it back as that language agnostic format. On Mon, Jun 29, 2015 at 6:39 AM Daniel Darabos daniel.dara...@lynxanalytics.com wrote: Hi Vasili, It so happens that the entire SparkR code was merged to Apache Spark in a single pull request. So you can see at once all the required changes in https://github.com/apache/spark/pull/5096. It's 12,043 lines and took more than 20 people about a year to write as I understand it. On Mon, Jun 29, 2015 at 10:33 AM, Vasili I. Galchin vigalc...@gmail.com wrote: Shivaram, Vis-a-vis Haskell support, I am reading DataFrame.R, SparkRBackend*, context.R, et. al., am I headed in the correct direction?/ Yes or no, please give more guidance. Thank you. Kind regards, Vasili On Tue, Jun 23, 2015 at 1:46 PM, Shivaram Venkataraman shiva...@eecs.berkeley.edu wrote: Every language has its own quirks / features -- so I don't think there exists a document on how to go about doing this for a new language. The most related write up I know of is the wiki page on PySpark internals https://cwiki.apache.org/confluence/display/SPARK/PySpark+Internals written by Josh Rosen -- It covers some of the issues like closure capture, serialization, JVM communication that you'll need to handle for a new language. Thanks Shivaram On Tue, Jun 23, 2015 at 1:35 PM, Vasili I. Galchin vigalc...@gmail.com wrote: Hello, I want to add language support for another language(other than Scala, Java et. al.). Where is documentation that explains to provide support for a new language? Thank you, Vasili - To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org For additional commands, e-mail: dev-h...@spark.apache.org