Re: Compiling Spark with a local hadoop profile

2015-10-09 Thread Steve Loughran

> On 8 Oct 2015, at 19:31, sbiookag  wrote:
> 
> Thanks Ted for reply.
> 
> But this is not what I want. This would tell spark to read hadoop dependency
> from maven repository, which is the original version of hadoop. I myslef is
> modifying the hadoop code, and wanted to include them inside the spark fat
> jar. "Spark-Class" would run slaves with the fat jar created in the assembly
> folder, and that jar does not contain my modified classes. 

it should if you have built a local hadoop version and done the -Phadoop-2.6 
-Dhadoop.version=2.8.0-SNAPSHOT

if you are rebuilding hadoop with an existing version number (e.g. 2.6.0, 
2.7.1) then maven may not actually be picking up your new code


> 
> Something that confuses me is, what spark includes the hadoop classes in
> it's built jar output? Isn't it supposed to go and read from the hadoop
> folder in each worker node?


There's a hadoop-provided profile which you can build with; this should leave 
the hadoop artifacts (and other stuff expected to be in the far-end's 
classpath) out of the assembly

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



Operations with cached RDD

2015-10-09 Thread Ulanov, Alexander
Dear Spark developers,

I am trying to understand how Spark UI displays operation with the cached RDD.

For example, the following code caches an rdd:
>> val rdd = sc.parallelize(1 to 5, 5).zipWithIndex.cache
>> rdd.count
The Jobs tab shows me that the RDD is evaluated:
: 1 count at :24  2015/10/09 16:15:430.4 s   
1/1
: 0 zipWithIndex at  :21 2015/10/09 16:15:380.6 s  
 1/1
An I can observe this rdd in the Storage tab of Spark UI:
: ZippedWithIndexRDD  Memory Deserialized 1x Replicated

Then I want to make an operation over the cached RDD. I run the following code:
>> val g = rdd.groupByKey()
>> g.count
The Jobs tab shows me a new Job:
: 2 count at :26
Inside this Job there are two stages:
: 3 count at :26 +details 2015/10/09 16:16:18   0.2 s   5/5
: 2 zipWithIndex at :21
It shows that zipWithIndex is executed again. It does not seem to be 
reasonable, because the rdd is cached, and zipWithIndex is already executed 
previously.

Could you explain why if I perform an operation followed by an action on a 
cached RDD, then the last operation in the lineage of the cached RDD is shown 
to be executed in the Spark UI?


Best regards, Alexander


passing a AbstractFunction1 to sparkContext().runJob instead of a Closure

2015-10-09 Thread Niranda Perera
hi all,

I want to run a job in the spark context and since I am running the system
in the java environment, I can not use a closure in
the sparkContext().runJob. Instead, I am passing an AbstractFunction1
extension.

while I get the jobs run without an issue, I constantly get the following
WARN message

TID: [-1234] [] [2015-10-06 04:39:43,387]  WARN
{org.apache.spark.util.ClosureCleaner} -  Expected a closure; got
org.wso2.carbon.analytics.spark.core.sources.AnalyticsWritingFunction
{org.apache.spark.util.ClosureCleaner}


I want to know what are the implications of this approach?
could this WARN cause issues in the functionality later on?

rgds
-- 
Niranda
@n1r44 
+94-71-554-8430
https://pythagoreanscript.wordpress.com/


Re: Scala 2.11 builds broken/ Can the PR build run also 2.11?

2015-10-09 Thread Patrick Wendell
I would push back slightly. The reason we have the PR builds taking so long
is death by a million small things that we add. Doing a full 2.11 compile
is order minutes... it's a nontrivial increase to the build times.

It doesn't seem that bad to me to go back post-hoc once in a while and fix
2.11 bugs when they come up. It's on the order of once or twice per release
and the typesafe guys keep a close eye on it (thanks!). Compare that to
literally thousands of PR runs and a few minutes every time, IMO it's not
worth it.

On Fri, Oct 9, 2015 at 3:31 PM, Hari Shreedharan 
wrote:

> +1, much better than having a new PR each time to fix something for
> scala-2.11 every time a patch breaks it.
>
> Thanks,
> Hari Shreedharan
>
>
>
>
> On Oct 9, 2015, at 11:47 AM, Michael Armbrust 
> wrote:
>
> How about just fixing the warning? I get it; it doesn't stop this from
>> happening again, but still seems less drastic than tossing out the
>> whole mechanism.
>>
>
> +1
>
> It also does not seem that expensive to test only compilation for Scala
> 2.11 on PR builds.
>
>
>


Re: Scala 2.11 builds broken/ Can the PR build run also 2.11?

2015-10-09 Thread Prashant Sharma
That is correct !, I have thought about this a lot of times. The only
solution is to implement a "real" cross build for both version. I am going
to think more in this. :)

Prashant Sharma



On Sat, Oct 10, 2015 at 2:04 AM, Patrick Wendell  wrote:

> I would push back slightly. The reason we have the PR builds taking so
> long is death by a million small things that we add. Doing a full 2.11
> compile is order minutes... it's a nontrivial increase to the build times.
>
> It doesn't seem that bad to me to go back post-hoc once in a while and fix
> 2.11 bugs when they come up. It's on the order of once or twice per release
> and the typesafe guys keep a close eye on it (thanks!). Compare that to
> literally thousands of PR runs and a few minutes every time, IMO it's not
> worth it.
>
> On Fri, Oct 9, 2015 at 3:31 PM, Hari Shreedharan <
> hshreedha...@cloudera.com> wrote:
>
>> +1, much better than having a new PR each time to fix something for
>> scala-2.11 every time a patch breaks it.
>>
>> Thanks,
>> Hari Shreedharan
>>
>>
>>
>>
>> On Oct 9, 2015, at 11:47 AM, Michael Armbrust 
>> wrote:
>>
>> How about just fixing the warning? I get it; it doesn't stop this from
>>> happening again, but still seems less drastic than tossing out the
>>> whole mechanism.
>>>
>>
>> +1
>>
>> It also does not seem that expensive to test only compilation for Scala
>> 2.11 on PR builds.
>>
>>
>>
>


sbt test error -- "Could not reserve enough space"

2015-10-09 Thread Robert Dodier
Hi,

I am trying to  build and test the current master. My system is Ubuntu
14.04 with 4 G physical memory with Oracle Java 8.

I have been running into various out-of-memory errors. I tried
building with Maven but couldn't get all the way through compile and
package. I'm having better luck with sbt. At this point build/sbt
package runs to completion, so that's great.

When I try to run build/sbt test, I get a lot of errors saying: "Could
not reserve enough space for 3145728KB object heap". Unfortunately 3.1
G is somewhat larger than the available memory, as reported by 'free'.
Is there any way to convince sbt that it needs to allocate less
memory?

I tried build/sbt "test-only
org.apache.spark.mllib.random.RandomDataGeneratorSuite" (I'm not
particularly interested in that test, it's just one that I thought
would be relatively simple) but it seems to do a lot more work than
just running that one test, and I still get the out-of-memory errors.

Aside from getting a machine with more memory (which is not out of the
question), are there any stretegies for coping with out-of-memory
errors in Maven and/or sbt?

Thanks in advance for any light you can shed on this problem.

Robert Dodier

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



Re: Scala 2.11 builds broken/ Can the PR build run also 2.11?

2015-10-09 Thread Iulian Dragoș
Sorry for not being clear, yes, that's about the Sbt build and treating
warnings as errors.

Warnings in 2.11 are useful, though, it'd be a pity to keep introducing
potential issues. As a stop-gap measure I can disable them in the Sbt
build, is it hard to run the CI test with 2.11/sbt?

iulian


On Thu, Oct 8, 2015 at 7:24 PM, Reynold Xin  wrote:

> The problem only applies to the sbt build because it treats warnings as
> errors.
>
> @Iulian - how about we disable warnings -> errors for 2.11? That would
> seem better until we switch 2.11 to be the default build.
>
>
> On Thu, Oct 8, 2015 at 7:55 AM, Ted Yu  wrote:
>
>> I tried building with Scala 2.11 on Linux with latest master branch :
>>
>> [INFO] Spark Project External MQTT  SUCCESS [
>> 19.188 s]
>> [INFO] Spark Project External MQTT Assembly ... SUCCESS [
>>  7.081 s]
>> [INFO] Spark Project External ZeroMQ .. SUCCESS [
>>  8.790 s]
>> [INFO] Spark Project External Kafka ... SUCCESS [
>> 14.764 s]
>> [INFO] Spark Project Examples . SUCCESS
>> [02:22 min]
>> [INFO] Spark Project External Kafka Assembly .. SUCCESS [
>> 10.286 s]
>> [INFO]
>> 
>> [INFO] BUILD SUCCESS
>> [INFO]
>> 
>> [INFO] Total time: 17:49 min
>>
>> FYI
>>
>> On Thu, Oct 8, 2015 at 6:50 AM, Ted Yu  wrote:
>>
>>> Interesting
>>>
>>>
>>> https://amplab.cs.berkeley.edu/jenkins/view/Spark-QA-Compile/job/Spark-Master-Scala211-Compile/
>>> shows green builds.
>>>
>>>
>>> On Thu, Oct 8, 2015 at 6:40 AM, Iulian Dragoș <
>>> iulian.dra...@typesafe.com> wrote:
>>>
 Since Oct. 4 the build fails on 2.11 with the dreaded

 [error] /home/ubuntu/workspace/Apache Spark (master) on 
 2.11/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala:310: 
 no valid targets for annotation on value conf - it is discarded unused. 
 You may specify targets with meta-annotations, e.g. @(transient @param)
 [error] private[netty] class NettyRpcEndpointRef(@transient conf: 
 SparkConf)

 Can we have the pull request builder at least build with 2.11? This
 makes #8433  pretty much
 useless, since people will continue to add useless @transient annotations.
 ​
 --

 --
 Iulian Dragos

 --
 Reactive Apps on the JVM
 www.typesafe.com


>>>
>>
>


-- 

--
Iulian Dragos

--
Reactive Apps on the JVM
www.typesafe.com


Re: Scala 2.11 builds broken/ Can the PR build run also 2.11?

2015-10-09 Thread Sean Owen
How about just fixing the warning? I get it; it doesn't stop this from
happening again, but still seems less drastic than tossing out the
whole mechanism.

On Fri, Oct 9, 2015 at 3:18 PM, Iulian Dragoș
 wrote:
> Sorry for not being clear, yes, that's about the Sbt build and treating
> warnings as errors.
>
> Warnings in 2.11 are useful, though, it'd be a pity to keep introducing
> potential issues. As a stop-gap measure I can disable them in the Sbt build,
> is it hard to run the CI test with 2.11/sbt?
>
> iulian
>

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



Re: Scala 2.11 builds broken/ Can the PR build run also 2.11?

2015-10-09 Thread Michael Armbrust
>
> How about just fixing the warning? I get it; it doesn't stop this from
> happening again, but still seems less drastic than tossing out the
> whole mechanism.
>

+1

It also does not seem that expensive to test only compilation for Scala
2.11 on PR builds.


Re: Scala 2.11 builds broken/ Can the PR build run also 2.11?

2015-10-09 Thread Hari Shreedharan
+1, much better than having a new PR each time to fix something for scala-2.11 
every time a patch breaks it.

Thanks,
Hari Shreedharan




> On Oct 9, 2015, at 11:47 AM, Michael Armbrust  wrote:
> 
> How about just fixing the warning? I get it; it doesn't stop this from
> happening again, but still seems less drastic than tossing out the
> whole mechanism.
> 
> +1
> 
> It also does not seem that expensive to test only compilation for Scala 2.11 
> on PR builds.