Re: [VOTE] Release Apache Spark 2.0.0 (RC1)

2016-06-23 Thread Yin Huai
-1 because of https://issues.apache.org/jira/browse/SPARK-16121.

This jira was resolved after 2.0.0-RC1 was cut. Without the fix, Spark
SQL effectively only uses the driver to list files when loading datasets
and the driver-side file listing is very slow for datasets having many
files and partitions. Since this bug causes a serious performance
regression, I am giving -1.

On Thu, Jun 23, 2016 at 1:25 AM, Pete Robbins  wrote:

> I'm also seeing some of these same failures:
>
> - spilling with compression *** FAILED ***
> I have seen this occassionaly
>
> - to UTC timestamp *** FAILED ***
> This was fixed yesterday in branch-2.0 (
> https://issues.apache.org/jira/browse/SPARK-16078)
>
> - offset recovery *** FAILED ***
> Haven't seen this for a while and thought the flaky test was fixed but it
> popped up again in one of our builds.
>
> StateStoreSuite:
> - maintenance *** FAILED ***
> Just seen this has been failing for last 2 days on one build machine
> (linux amd64)
>
>
> On 23 June 2016 at 08:51, Sean Owen  wrote:
>
>> First pass of feedback on the RC: all the sigs, hashes, etc are fine.
>> Licensing is up to date to the best of my knowledge.
>>
>> I'm hitting test failures, some of which may be spurious. Just putting
>> them out there to see if they ring bells. This is Java 8 on Ubuntu 16.
>>
>>
>> - spilling with compression *** FAILED ***
>>   java.lang.Exception: Test failed with compression using codec
>> org.apache.spark.io.SnappyCompressionCodec:
>> assertion failed: expected cogroup to spill, but did not
>>   at scala.Predef$.assert(Predef.scala:170)
>>   at org.apache.spark.TestUtils$.assertSpilled(TestUtils.scala:170)
>>   at org.apache.spark.util.collection.ExternalAppendOnlyMapSuite.org
>> $apache$spark$util$collection$ExternalAppendOnlyMapSuite$$testSimpleSpilling(ExternalAppendOnlyMapSuite.scala:263)
>> ...
>>
>> I feel like I've seen this before, and see some possibly relevant
>> fixes, but they're in 2.0.0 already:
>> https://github.com/apache/spark/pull/10990
>> Is this something where a native library needs to be installed or
>> something?
>>
>>
>> - to UTC timestamp *** FAILED ***
>>   "2016-03-13 [02]:00:00.0" did not equal "2016-03-13 [10]:00:00.0"
>> (DateTimeUtilsSuite.scala:506)
>>
>> I know, we talked about this for the 1.6.2 RC, but I reproduced this
>> locally too. I will investigate, could still be spurious.
>>
>>
>> StateStoreSuite:
>> - maintenance *** FAILED ***
>>   The code passed to eventually never returned normally. Attempted 627
>> times over 10.000180116 seconds. Last failure message:
>> StateStoreSuite.this.fileExists(provider, 1L, false) was true earliest
>> file not deleted. (StateStoreSuite.scala:395)
>>
>> No idea.
>>
>>
>> - offset recovery *** FAILED ***
>>   The code passed to eventually never returned normally. Attempted 197
>> times over 10.040864806 seconds. Last failure message:
>> strings.forall({
>> ((x$1: Any) => DirectKafkaStreamSuite.collectedData.contains(x$1))
>>   }) was false. (DirectKafkaStreamSuite.scala:250)
>>
>> Also something that was possibly fixed already for 2.0.0 and that I
>> just back-ported into 1.6. Could be just a very similar failure.
>>
>> On Wed, Jun 22, 2016 at 2:26 AM, Reynold Xin  wrote:
>> > Please vote on releasing the following candidate as Apache Spark version
>> > 2.0.0. The vote is open until Friday, June 24, 2016 at 19:00 PDT and
>> passes
>> > if a majority of at least 3+1 PMC votes are cast.
>> >
>> > [ ] +1 Release this package as Apache Spark 2.0.0
>> > [ ] -1 Do not release this package because ...
>> >
>> >
>> > The tag to be voted on is v2.0.0-rc1
>> > (0c66ca41afade6db73c9aeddd5aed6e5dcea90df).
>> >
>> > This release candidate resolves ~2400 issues:
>> > https://s.apache.org/spark-2.0.0-rc1-jira
>> >
>> > The release files, including signatures, digests, etc. can be found at:
>> > http://people.apache.org/~pwendell/spark-releases/spark-2.0.0-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:
>> > https://repository.apache.org/content/repositories/orgapachespark-1187/
>> >
>> > The documentation corresponding to this release can be found at:
>> > http://people.apache.org/~pwendell/spark-releases/spark-2.0.0-rc1-docs/
>> >
>> >
>> > ===
>> > == How can I help test this release? ==
>> > ===
>> > If you are a Spark user, you can help us test this release by taking an
>> > existing Spark workload and running on this release candidate, then
>> > reporting any regressions from 1.x.
>> >
>> > 
>> > == What justifies a -1 vote for this release? ==
>> > 
>> > Critical bugs impacting major functionalities.
>> >
>> > Bugs already present in 1.x, 

destroyPythonWorker job in PySpark

2016-06-23 Thread Krishna
Hi,

I am running a PySpark app with 1000's of cores (partitions is a small
multiple of # of cores) and the overall application performance is fine.
However, I noticed that, at the end of the job, PySpark initiates job
clean-up procedures and as part of this procedure, PySpark executes a job
shown in the Web UI as "runJob at PythonRDD.scala:361" for each
executor/core. The pain point is that, this step is running in a sequential
fashion and it has become the bottleneck in our application. Even though
each job takes only 0.5 sec (on average), it adds up when running with
1000's of executors.

Looking into the code for "destroyPythonWorker" in "SparkEnv.scala", is
this behavior the result of "stopWorker" being executed sequentially within
foreach? Let me know if I'm missing something and what can be done to fix
the issue.

  private[spark]
>   def destroyPythonWorker(pythonExec: String, envVars: Map[String,
> String], worker: Socket) {
> synchronized {
>   val key = (pythonExec, envVars)
>   pythonWorkers.get(key).foreach(_.stopWorker(worker))
> }
>   }



Spark version: 1.5.0-cdh5.5.1

Thanks


Does CoarseGrainedSchedulerBackend care about cores only? And disregards memory?

2016-06-23 Thread Jacek Laskowski
Hi,

After reviewing makeOffer and launchTasks in
CoarseGrainedSchedulerBackend I came to the following conclusion:

Scheduling in Spark relies on cores only (not memory), i.e. the number
of tasks Spark can run on an executor is constrained by the number of
cores available only. When submitting Spark application for execution
both -- memory and cores -- can be specified explicitly.

Would you agree? Do I miss anything important?

I was very surprised when I found it out as I thought that memory
would also have been a limiting factor.

Pozdrawiam,
Jacek Laskowski

https://medium.com/@jaceklaskowski/
Mastering Apache Spark http://bit.ly/mastering-apache-spark
Follow me at https://twitter.com/jaceklaskowski

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



Re: Spark Thrift Server Concurrency

2016-06-23 Thread Michael Segel
Hi, 
There are  a lot of moving parts and a lot of unknowns from your description. 
Besides the version stuff. 

How many executors, how many cores? How much memory? 
Are you persisting (memory and disk) or just caching (memory) 

During the execution… same tables… are  you seeing a lot of shuffling of data 
for some queries and not others? 

It sounds like an interesting problem… 

> On Jun 23, 2016, at 5:21 AM, Prabhu Joseph  wrote:
> 
> Hi All,
> 
>On submitting 20 parallel same SQL query to Spark Thrift Server, the query 
> execution time for some queries are less than a second and some are more than 
> 2seconds. The Spark Thrift Server logs shows all 20 queries are submitted at 
> same time 16/06/23 12:12:01 but the result schema are at different times.
> 
> 16/06/23 12:12:01 INFO SparkExecuteStatementOperation: Running query 'select 
> distinct val2 from philips1 where key>=1000 and key<=1500
> 
> 16/06/23 12:12:02 INFO SparkExecuteStatementOperation: Result Schema: 
> ArrayBuffer(val2#2110)
> 16/06/23 12:12:03 INFO SparkExecuteStatementOperation: Result Schema: 
> ArrayBuffer(val2#2182)
> 16/06/23 12:12:04 INFO SparkExecuteStatementOperation: Result Schema: 
> ArrayBuffer(val2#2344)
> 16/06/23 12:12:05 INFO SparkExecuteStatementOperation: Result Schema: 
> ArrayBuffer(val2#2362)
> 
> There are sufficient executors running on YARN. The concurrency is affected 
> by Single Driver. How to improve the concurrency and what are the best 
> practices.
> 
> Thanks,
> Prabhu Joseph



Re: [VOTE][RESULT] Release Apache Spark 1.6.2 (RC2)

2016-06-23 Thread Reynold Xin
Vote passed. Please see below. I will work on packaging the release.

+1 (9 votes, 4 binding)
Reynold Xin*
Sean Owen*
Tim Hunter
Michael Armbrust*
Sean McNamara*
Kousuke Saruta
Sameer Agarwal
Krishna Sankar
Vaquar Khan

0
none

-1
Maciej Bryński


* binding votes


On Sun, Jun 19, 2016 at 9:24 PM, Reynold Xin  wrote:

> Please vote on releasing the following candidate as Apache Spark version
> 1.6.2. The vote is open until Wednesday, June 22, 2016 at 22:00 PDT and
> passes if a majority of at least 3+1 PMC votes are cast.
>
> [ ] +1 Release this package as Apache Spark 1.6.2
> [ ] -1 Do not release this package because ...
>
>
> The tag to be voted on is v1.6.2-rc2
> (54b1121f351f056d6b67d2bb4efe0d553c0f7482)
>
> The release files, including signatures, digests, etc. can be found at:
> http://people.apache.org/~pwendell/spark-releases/spark-1.6.2-rc2-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:
> https://repository.apache.org/content/repositories/orgapachespark-1186/
>
> The documentation corresponding to this release can be found at:
> http://people.apache.org/~pwendell/spark-releases/spark-1.6.2-rc2-docs/
>
>
> ===
> == How can I help test this release? ==
> ===
> If you are a Spark user, you can help us test this release by taking an
> existing Spark workload and running on this release candidate, then
> reporting any regressions from 1.6.1.
>
> 
> == What justifies a -1 vote for this release? ==
> 
> This is a maintenance release in the 1.6.x series.  Bugs already present
> in 1.6.1, missing features, or bugs related to new features will not
> necessarily block this release.
>
>
>
>


Re: [VOTE] Release Apache Spark 1.6.2 (RC2)

2016-06-23 Thread Reynold Xin
Maciej let's fix SPARK-13283. It won't block 1.6.2 though.

On Thu, Jun 23, 2016 at 5:45 AM, Maciej Bryński  wrote:

> -1
>
> I need SPARK-13283 to be solved.
>
> Regards,
> Maciek Bryński
>
> 2016-06-23 0:13 GMT+02:00 Krishna Sankar :
>
>> +1 (non-binding, of course)
>>
>> 1. Compiled OSX 10.10 (Yosemite) OK Total time: 37:11 min
>>  mvn clean package -Pyarn -Phadoop-2.6 -DskipTests
>> 2. Tested pyspark, mllib (iPython 4.0)
>> 2.0 Spark version is 1.6.2
>> 2.1. statistics (min,max,mean,Pearson,Spearman) OK
>> 2.2. Linear/Ridge/Lasso 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 (--packages
>> com.databricks:spark-csv_2.10:1.4.0)
>> 6.0. DataFrames
>> 6.1. cast,dtypes OK
>> 6.2. groupBy,avg,crosstab,corr,isNull,na.drop OK
>> 6.3. All joins,sql,set operations,udf OK
>> 7.0. GraphX/Scala
>> 7.1. Create Graph (small and bigger dataset) OK
>> 7.2. Structure APIs - OK
>> 7.3. Social Network/Community APIs - OK
>> 7.4. Algorithms (PageRank of 2 datasets, aggregateMessages() ) OK
>>
>> Cheers & Good Work, Folks
>> 
>>
>> On Sun, Jun 19, 2016 at 9:24 PM, Reynold Xin  wrote:
>>
>>> Please vote on releasing the following candidate as Apache Spark version
>>> 1.6.2. The vote is open until Wednesday, June 22, 2016 at 22:00 PDT and
>>> passes if a majority of at least 3+1 PMC votes are cast.
>>>
>>> [ ] +1 Release this package as Apache Spark 1.6.2
>>> [ ] -1 Do not release this package because ...
>>>
>>>
>>> The tag to be voted on is v1.6.2-rc2
>>> (54b1121f351f056d6b67d2bb4efe0d553c0f7482)
>>>
>>> The release files, including signatures, digests, etc. can be found at:
>>> http://people.apache.org/~pwendell/spark-releases/spark-1.6.2-rc2-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:
>>> https://repository.apache.org/content/repositories/orgapachespark-1186/
>>>
>>> The documentation corresponding to this release can be found at:
>>> http://people.apache.org/~pwendell/spark-releases/spark-1.6.2-rc2-docs/
>>>
>>>
>>> ===
>>> == How can I help test this release? ==
>>> ===
>>> If you are a Spark user, you can help us test this release by taking an
>>> existing Spark workload and running on this release candidate, then
>>> reporting any regressions from 1.6.1.
>>>
>>> 
>>> == What justifies a -1 vote for this release? ==
>>> 
>>> This is a maintenance release in the 1.6.x series.  Bugs already present
>>> in 1.6.1, missing features, or bugs related to new features will not
>>> necessarily block this release.
>>>
>>>
>>>
>>>
>>
>
>
> --
> Maciek Bryński
>


Re: [VOTE] Release Apache Spark 1.6.2 (RC2)

2016-06-23 Thread vaquar khan
+1 (non-binding

Regards,
Vaquar khan
On 23 Jun 2016 07:50, "Sean Owen"  wrote:

> I don't think that qualifies as a blocker; not even clear it's a
> regression. Even non-binding votes here should focus on whether this
> is OK to release as a maintenance update to 1.6.1.
>
> On Thu, Jun 23, 2016 at 1:45 PM, Maciej Bryński  wrote:
> > -1
> >
> > I need SPARK-13283 to be solved.
> >
> > Regards,
> > Maciek Bryński
> >
>
> -
> 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.6.2 (RC2)

2016-06-23 Thread Sean Owen
I don't think that qualifies as a blocker; not even clear it's a
regression. Even non-binding votes here should focus on whether this
is OK to release as a maintenance update to 1.6.1.

On Thu, Jun 23, 2016 at 1:45 PM, Maciej Bryński  wrote:
> -1
>
> I need SPARK-13283 to be solved.
>
> Regards,
> Maciek Bryński
>

-
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.6.2 (RC2)

2016-06-23 Thread Maciej Bryński
-1

I need SPARK-13283 to be solved.

Regards,
Maciek Bryński

2016-06-23 0:13 GMT+02:00 Krishna Sankar :

> +1 (non-binding, of course)
>
> 1. Compiled OSX 10.10 (Yosemite) OK Total time: 37:11 min
>  mvn clean package -Pyarn -Phadoop-2.6 -DskipTests
> 2. Tested pyspark, mllib (iPython 4.0)
> 2.0 Spark version is 1.6.2
> 2.1. statistics (min,max,mean,Pearson,Spearman) OK
> 2.2. Linear/Ridge/Lasso 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 (--packages
> com.databricks:spark-csv_2.10:1.4.0)
> 6.0. DataFrames
> 6.1. cast,dtypes OK
> 6.2. groupBy,avg,crosstab,corr,isNull,na.drop OK
> 6.3. All joins,sql,set operations,udf OK
> 7.0. GraphX/Scala
> 7.1. Create Graph (small and bigger dataset) OK
> 7.2. Structure APIs - OK
> 7.3. Social Network/Community APIs - OK
> 7.4. Algorithms (PageRank of 2 datasets, aggregateMessages() ) OK
>
> Cheers & Good Work, Folks
> 
>
> On Sun, Jun 19, 2016 at 9:24 PM, Reynold Xin  wrote:
>
>> Please vote on releasing the following candidate as Apache Spark version
>> 1.6.2. The vote is open until Wednesday, June 22, 2016 at 22:00 PDT and
>> passes if a majority of at least 3+1 PMC votes are cast.
>>
>> [ ] +1 Release this package as Apache Spark 1.6.2
>> [ ] -1 Do not release this package because ...
>>
>>
>> The tag to be voted on is v1.6.2-rc2
>> (54b1121f351f056d6b67d2bb4efe0d553c0f7482)
>>
>> The release files, including signatures, digests, etc. can be found at:
>> http://people.apache.org/~pwendell/spark-releases/spark-1.6.2-rc2-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:
>> https://repository.apache.org/content/repositories/orgapachespark-1186/
>>
>> The documentation corresponding to this release can be found at:
>> http://people.apache.org/~pwendell/spark-releases/spark-1.6.2-rc2-docs/
>>
>>
>> ===
>> == How can I help test this release? ==
>> ===
>> If you are a Spark user, you can help us test this release by taking an
>> existing Spark workload and running on this release candidate, then
>> reporting any regressions from 1.6.1.
>>
>> 
>> == What justifies a -1 vote for this release? ==
>> 
>> This is a maintenance release in the 1.6.x series.  Bugs already present
>> in 1.6.1, missing features, or bugs related to new features will not
>> necessarily block this release.
>>
>>
>>
>>
>


-- 
Maciek Bryński


Spark Thrift Server Concurrency

2016-06-23 Thread Prabhu Joseph
Hi All,

   On submitting 20 parallel same SQL query to Spark Thrift Server, the
query execution time for some queries are less than a second and some are
more than 2seconds. The Spark Thrift Server logs shows all 20 queries are
submitted at same time 16/06/23 12:12:01 but the result schema are at
different times.

16/06/23 12:12:01 INFO SparkExecuteStatementOperation: Running query
'select distinct val2 from philips1 where key>=1000 and key<=1500

16/06/23 12:12:*02* INFO SparkExecuteStatementOperation: Result Schema:
ArrayBuffer(val2#2110)
16/06/23 12:12:*03* INFO SparkExecuteStatementOperation: Result Schema:
ArrayBuffer(val2#2182)
16/06/23 12:12:*04* INFO SparkExecuteStatementOperation: Result Schema:
ArrayBuffer(val2#2344)
16/06/23 12:12:*05* INFO SparkExecuteStatementOperation: Result Schema:
ArrayBuffer(val2#2362)

There are sufficient executors running on YARN. The concurrency is affected
by Single Driver. How to improve the concurrency and what are the best
practices.

Thanks,
Prabhu Joseph


Re: [VOTE] Release Apache Spark 2.0.0 (RC1)

2016-06-23 Thread Pete Robbins
I'm also seeing some of these same failures:

- spilling with compression *** FAILED ***
I have seen this occassionaly

- to UTC timestamp *** FAILED ***
This was fixed yesterday in branch-2.0 (
https://issues.apache.org/jira/browse/SPARK-16078)

- offset recovery *** FAILED ***
Haven't seen this for a while and thought the flaky test was fixed but it
popped up again in one of our builds.

StateStoreSuite:
- maintenance *** FAILED ***
Just seen this has been failing for last 2 days on one build machine (linux
amd64)

On 23 June 2016 at 08:51, Sean Owen  wrote:

> First pass of feedback on the RC: all the sigs, hashes, etc are fine.
> Licensing is up to date to the best of my knowledge.
>
> I'm hitting test failures, some of which may be spurious. Just putting
> them out there to see if they ring bells. This is Java 8 on Ubuntu 16.
>
>
> - spilling with compression *** FAILED ***
>   java.lang.Exception: Test failed with compression using codec
> org.apache.spark.io.SnappyCompressionCodec:
> assertion failed: expected cogroup to spill, but did not
>   at scala.Predef$.assert(Predef.scala:170)
>   at org.apache.spark.TestUtils$.assertSpilled(TestUtils.scala:170)
>   at org.apache.spark.util.collection.ExternalAppendOnlyMapSuite.org
> $apache$spark$util$collection$ExternalAppendOnlyMapSuite$$testSimpleSpilling(ExternalAppendOnlyMapSuite.scala:263)
> ...
>
> I feel like I've seen this before, and see some possibly relevant
> fixes, but they're in 2.0.0 already:
> https://github.com/apache/spark/pull/10990
> Is this something where a native library needs to be installed or
> something?
>
>
> - to UTC timestamp *** FAILED ***
>   "2016-03-13 [02]:00:00.0" did not equal "2016-03-13 [10]:00:00.0"
> (DateTimeUtilsSuite.scala:506)
>
> I know, we talked about this for the 1.6.2 RC, but I reproduced this
> locally too. I will investigate, could still be spurious.
>
>
> StateStoreSuite:
> - maintenance *** FAILED ***
>   The code passed to eventually never returned normally. Attempted 627
> times over 10.000180116 seconds. Last failure message:
> StateStoreSuite.this.fileExists(provider, 1L, false) was true earliest
> file not deleted. (StateStoreSuite.scala:395)
>
> No idea.
>
>
> - offset recovery *** FAILED ***
>   The code passed to eventually never returned normally. Attempted 197
> times over 10.040864806 seconds. Last failure message:
> strings.forall({
> ((x$1: Any) => DirectKafkaStreamSuite.collectedData.contains(x$1))
>   }) was false. (DirectKafkaStreamSuite.scala:250)
>
> Also something that was possibly fixed already for 2.0.0 and that I
> just back-ported into 1.6. Could be just a very similar failure.
>
> On Wed, Jun 22, 2016 at 2:26 AM, Reynold Xin  wrote:
> > Please vote on releasing the following candidate as Apache Spark version
> > 2.0.0. The vote is open until Friday, June 24, 2016 at 19:00 PDT and
> passes
> > if a majority of at least 3+1 PMC votes are cast.
> >
> > [ ] +1 Release this package as Apache Spark 2.0.0
> > [ ] -1 Do not release this package because ...
> >
> >
> > The tag to be voted on is v2.0.0-rc1
> > (0c66ca41afade6db73c9aeddd5aed6e5dcea90df).
> >
> > This release candidate resolves ~2400 issues:
> > https://s.apache.org/spark-2.0.0-rc1-jira
> >
> > The release files, including signatures, digests, etc. can be found at:
> > http://people.apache.org/~pwendell/spark-releases/spark-2.0.0-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:
> > https://repository.apache.org/content/repositories/orgapachespark-1187/
> >
> > The documentation corresponding to this release can be found at:
> > http://people.apache.org/~pwendell/spark-releases/spark-2.0.0-rc1-docs/
> >
> >
> > ===
> > == How can I help test this release? ==
> > ===
> > If you are a Spark user, you can help us test this release by taking an
> > existing Spark workload and running on this release candidate, then
> > reporting any regressions from 1.x.
> >
> > 
> > == What justifies a -1 vote for this release? ==
> > 
> > Critical bugs impacting major functionalities.
> >
> > Bugs already present in 1.x, missing features, or bugs related to new
> > features will not necessarily block this release. Note that historically
> > Spark documentation has been published on the website separately from the
> > main release so we do not need to block the release due to documentation
> > errors either.
> >
> >
>
> -
> To unsubscribe, e-mail: dev-unsubscr...@spark.apache.org
> For additional commands, e-mail: dev-h...@spark.apache.org
>
>


Re: [VOTE] Release Apache Spark 2.0.0 (RC1)

2016-06-23 Thread Sean Owen
First pass of feedback on the RC: all the sigs, hashes, etc are fine.
Licensing is up to date to the best of my knowledge.

I'm hitting test failures, some of which may be spurious. Just putting
them out there to see if they ring bells. This is Java 8 on Ubuntu 16.


- spilling with compression *** FAILED ***
  java.lang.Exception: Test failed with compression using codec
org.apache.spark.io.SnappyCompressionCodec:
assertion failed: expected cogroup to spill, but did not
  at scala.Predef$.assert(Predef.scala:170)
  at org.apache.spark.TestUtils$.assertSpilled(TestUtils.scala:170)
  at 
org.apache.spark.util.collection.ExternalAppendOnlyMapSuite.org$apache$spark$util$collection$ExternalAppendOnlyMapSuite$$testSimpleSpilling(ExternalAppendOnlyMapSuite.scala:263)
...

I feel like I've seen this before, and see some possibly relevant
fixes, but they're in 2.0.0 already:
https://github.com/apache/spark/pull/10990
Is this something where a native library needs to be installed or something?


- to UTC timestamp *** FAILED ***
  "2016-03-13 [02]:00:00.0" did not equal "2016-03-13 [10]:00:00.0"
(DateTimeUtilsSuite.scala:506)

I know, we talked about this for the 1.6.2 RC, but I reproduced this
locally too. I will investigate, could still be spurious.


StateStoreSuite:
- maintenance *** FAILED ***
  The code passed to eventually never returned normally. Attempted 627
times over 10.000180116 seconds. Last failure message:
StateStoreSuite.this.fileExists(provider, 1L, false) was true earliest
file not deleted. (StateStoreSuite.scala:395)

No idea.


- offset recovery *** FAILED ***
  The code passed to eventually never returned normally. Attempted 197
times over 10.040864806 seconds. Last failure message:
strings.forall({
((x$1: Any) => DirectKafkaStreamSuite.collectedData.contains(x$1))
  }) was false. (DirectKafkaStreamSuite.scala:250)

Also something that was possibly fixed already for 2.0.0 and that I
just back-ported into 1.6. Could be just a very similar failure.

On Wed, Jun 22, 2016 at 2:26 AM, Reynold Xin  wrote:
> Please vote on releasing the following candidate as Apache Spark version
> 2.0.0. The vote is open until Friday, June 24, 2016 at 19:00 PDT and passes
> if a majority of at least 3+1 PMC votes are cast.
>
> [ ] +1 Release this package as Apache Spark 2.0.0
> [ ] -1 Do not release this package because ...
>
>
> The tag to be voted on is v2.0.0-rc1
> (0c66ca41afade6db73c9aeddd5aed6e5dcea90df).
>
> This release candidate resolves ~2400 issues:
> https://s.apache.org/spark-2.0.0-rc1-jira
>
> The release files, including signatures, digests, etc. can be found at:
> http://people.apache.org/~pwendell/spark-releases/spark-2.0.0-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:
> https://repository.apache.org/content/repositories/orgapachespark-1187/
>
> The documentation corresponding to this release can be found at:
> http://people.apache.org/~pwendell/spark-releases/spark-2.0.0-rc1-docs/
>
>
> ===
> == How can I help test this release? ==
> ===
> If you are a Spark user, you can help us test this release by taking an
> existing Spark workload and running on this release candidate, then
> reporting any regressions from 1.x.
>
> 
> == What justifies a -1 vote for this release? ==
> 
> Critical bugs impacting major functionalities.
>
> Bugs already present in 1.x, missing features, or bugs related to new
> features will not necessarily block this release. Note that historically
> Spark documentation has been published on the website separately from the
> main release so we do not need to block the release due to documentation
> errors either.
>
>

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