Hi,
I was hoping to have some discussion about how sparse matrices are
represented in MLLib. I noticed a few commits to add a basic MatrixEntry
object:
https://github.com/apache/incubator-spark/blob/master/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala
I know that MatrixEn
The way to do this that comes to my mind is to use do two passes. The
first pass just builds up map from a partition to its first element. Then
you broadcast that map, and the second pass does what you want, using the
broadcast map to do what you want. Something like (pseudocode)
val rdd =...
v
The problem is that at compile time, sc.objectFile() has no idea what type
of objects its loading. Note the type of loadedFile:
loadedFile: org.apache.spark.rdd.RDD[Nothing]
that "Nothing" basically means the scala compiler has no idea what the type
of objects in the RDD are.
So, then when you
pRDD. It would be a cool feature but HDFS
> itself has no information about partitioning, so your application needs to
> track it.
>
> Matei
>
> On Jan 27, 2014, at 11:57 PM, Imran Rashid wrote:
>
> > Hi,
> >
> >
> > I'm trying to figure out how
Hi,
I'm trying to figure out how to get partitioners to work correctly with
hadoop rdds, so that I can get narrow dependencies & avoid shuffling. I
feel like I must be missing something obvious.
I can create an RDD with a parititioner of my choosing, shuffle it and then
save it out to hdfs. Bu
if I understand your issue correctly, there is an open PR to deal w/ this
issue:
https://github.com/apache/incubator-spark/pull/192
though I dunno if its gonna get merged or not ...
On Thu, Jan 23, 2014 at 6:16 PM, kyocum wrote:
> Trying to run spark-shell from my laptop to a master node in a
actually, I think adding it to SPARK_CLASSPATH is exactly right. The
exception is not on the executors, but in the driver -- its happening when
the driver tries to read results that the executor is sending back to it.
So the executors know about mypackage.MyClass, they happily run and send
their
nice work tracking down the problems w/ the codec getting applied
consistently. I think you're close to the fix, just need to understand
scala implicit resolution rules.
I'm not entirely sure what you mean when you say "I simply copy/pasted
saveAsObject() body to my funtion:" -- where does your f
I think a lot of the confusion is cleared up with a quick look at the code:
https://github.com/apache/incubator-spark/blob/master/core/src/main/scala/org/apache/spark/rdd/RDD.scala#L901
saveAsObjectFile is just a thin wrapper around saveAsSequenceFile, which
makes a null key and calls the java se
there is a separate setting for serializing closures
"spark.closure.serializer" (listed here
http://spark.incubator.apache.org/docs/latest/configuration.html)
that is used to serialize whatever is used by all the fucntions on an RDD,
eg., map, filter, and lookup. Those closures include referenced
alternative strategy that could allow you to get vectorized ops in
>> spark while still operating in a distributed fashion is to pack your
>> partitions into something like a jblas DoubleMatrix, one per partition,
>> where each partition is a contiguous block of elements. Then you c
these are just thoughts off the top of my head:
1) if the original R code runs in 3 secs, you are unlikely to be able to
improve that drastically with spark. Yes, spark can run sub-second jobs,
but no matter what, don't expect spark to get you into the < 10 millisecond
range. While spark has muc
btw, there is an open PR to allow spreadOut to be configured per-app,
instead of per-cluster
https://github.com/apache/incubator-spark/pull/136
On Tue, Nov 19, 2013 at 11:20 AM, Mark Hamstra wrote:
> No, it's my fault for not reading more carefully. We do use a somewhat
> overloaded and special
it show -p 3d52ea381cf53346f20c8558ce67cccbf6b361d7
commit 3d52ea381cf53346f20c8558ce67cccbf6b361d7
Author: Imran Rashid
Date: Tue Oct 29 08:21:55 2013 -0500
fix jetty orbit dep
diff --git a/project/Build.scala b/project/Build.scala
index 9f1cf03..a1057c9 100644
--- a/project/Build.scala
+++ b/project/Build.scala
o I jumped to conclusions.
thanks everyone
On Fri, Nov 1, 2013 at 9:27 AM, Imran Rashid wrote:
> thanks everyone for all of the input.
>
> Matei: makes a lot more sense with your explanation of spark's expected
> behavior of tcp, I can see why this makes sense now. But, to sho
tried tweaking various settings of akka(wrt heartbeats, failure
>> detector, even plugged-in our own failure detector with no effect).
>>
>> Imran - Just to clarify your point on message delivery - akka's message
>> delivery policy is at-most-once. However, there's
ilure detection timeouts.
>
> Matei
>
> On Oct 31, 2013, at 1:33 PM, Imran Rashid wrote:
>
> pretty sure I found the problem -- two problems actually. And I think one
> of them has been a general lurking problem w/ spark for a while.
>
> 1) we should ignore disass
nfo("Executor " + executorId + " disconnected, so removing
> it")
> +logInfo("Executor " + executorId + " disconnected, so removing
> it, reason:" + reason)
> val numCores = freeCores(executorId)
>
ot;)
> +logInfo("Executor " + executorId + " disconnected, so removing
> it, reason:" + reason)
> val numCores = freeCores(executorId)
> actorToExecutorId -= executorActor(executorId)
> addressToExecutorId -= executorAddress(executorId)
>
- so I'm pretty confident its
not a problem w/ the code. But maybe we need to add something like a
periodic heartbeat in this kind of operation?
On Wed, Oct 30, 2013 at 8:56 AM, Imran Rashid wrote:
> I'm gonna try turning on more akka debugging msgs as described at
> http://akk
ection timeouts,
> setting those properties should work, but I am afraid about Disassociation
> Event though. I will have to check this is indeed hard to reproduce bug if
> it is, I mean how do I simulate network delays ?
>
>
> On Wed, Oct 30, 2013 at 6:05 PM, Imran Rashid wrote:
>
&g
;> It might be a temporary thing in that case.
>>
>> I had like to have more details on the long jobs though, how long ?
>>
>>
>> On Wed, Oct 30, 2013 at 1:29 AM, Imran Rashid wrote:
>>
>>> We've been testing out the 2.10 branch of spark, and we'
We've been testing out the 2.10 branch of spark, and we're running into
some issues were akka disconnects from the executors after a while. We ran
some simple tests first, and all was well, so we started upgrading our
whole codebase to 2.10. Everything seemed to be working, but then we
noticed th
23 matches
Mail list logo