Dachuan,
Where did you find that faulty documentation? I'd like to get it fixed.
Thanks!
Andrew
On Tue, Feb 18, 2014 at 4:15 PM, dachuan wrote:
> Thanks for your reply.
>
> I have changed scalaVersion := "2.10" to scalaVersion := "2.10.3" then
> everything is good.
>
> So this is a documenta
Why would scala 0.11 change things here? I'm not familiar with what
features you're referring.
I would support a prelude file in ~/.sparkrc our similar that is
automatically imported on spark shell startup if it exists.
Sent from my mobile phone
On Feb 17, 2014 9:11 PM, "Prashant Sharma" wrote:
25 AM, Vyacheslav Baranov <
slavik.bara...@gmail.com> wrote:
> Andrew,
>
> I've created account on Amplab Jira, but unfortunately I don't have
> permission to comment.
>
> Vyacheslav
>
>
> On 15/02/14 00:28, Andrew Ash wrote:
>
> Hi Vyacheslav,
>
&
ubating-hadoop2.2.0.jar:/path/to/msgpack-0.6.8.jar"
> Nothing to replay.
>
> scala> import org.msgpack
> :7: error: object msgpack is not a member of package org
>import org.msgpack
> ^
>
> Probably, it's worth to add this to issue's commen
I filed a bug so we can track the fix:
https://spark-project.atlassian.net/browse/SPARK-1089
On Thu, Feb 13, 2014 at 2:21 PM, Soumya Simanta wrote:
> Use
> SPARK_CLASSPATH along with ADD_JARS
>
>
> On Thu, Feb 13, 2014 at 5:12 PM, Andre Kuhnen wrote:
>
>> Hello, my spark-shell tells me taht the
Hi Andre,
I've also noticed this. The jar needs to be added to SPARK_CLASSPATH also
now.
See
https://mail-archives.apache.org/mod_mbox/incubator-spark-user/201402.mbox/%3ccajbo4nemlitrnm1xbyqomwmp0m+eucg4ye-txurgsvkob5k...@mail.gmail.com%3E
On Thu, Feb 13, 2014 at 2:12 PM, Andre Kuhnen wrote:
Hi Egor,
It sounds like you should vote for
https://spark-project.atlassian.net/browse/SPARK-914 which is to make an
RDD iterable from the driver.
On Wed, Feb 12, 2014 at 1:07 AM, Egor Pahomov wrote:
> Hello. I've got big RDD(1gb) in yarn cluster. On local machine, which use
> this cluster I ha
Alternatively, Spark's estimate of how much space you're using in the heap
is off on the low-side of true, so it runs out of memory when it thinks it
has breathing room.
Try lowering spark.storage.memoryFraction from its default (0.6) a bit to
something like 0.5 to make it more conservative with m
Here's the original paper on how the framework achieves fault tolerance.
You shouldn't have to do anything special as a user of the framework.
https://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf
On Tue, Feb 11, 2014 at 12:21 PM, Adrian Mocanu
wrote:
> Anyone willing to link some res
Hi Ravi,
Have you read through the docs? I'm not sure there's a page that directly
answers your question but this one gives you an overview of the cluster.
http://spark.incubator.apache.org/docs/latest/cluster-overview.html
Andrew
On Tue, Feb 11, 2014 at 8:31 AM, Ravi Hemnani wrote:
> Hey,
>
>
>
> On Tue, Feb 11, 2014 at 10:35 AM, Andrew Ash wrote:
>
>> Do you want the files scattered across the local temp directories of all
>> your machines or just one of them? If just one, I'd recommend having your
>> driver program execute hadoop fs -getmerge /pat
Do you want the files scattered across the local temp directories of all
your machines or just one of them? If just one, I'd recommend having your
driver program execute hadoop fs -getmerge /path/to/files... using Scala's
external process libraries.
On Tue, Feb 11, 2014 at 9:18 AM, David Thomas
I would key by those things that should be the same and then reduce by sum.
sc.parallelize(inputList)
.map(x => (x._1, x._2.toLong, x._3.toLong)) // parse to numeric values from
String
.map(x => ((x._1, x._3), x._2)) // key by the name and final number field
.reduceByKey(_+_)
Andrew
On Tue, Feb
w occasions recently I had to lower both
> spark.storage.memoryFraction and spark.shuffle.memoryFraction
> spark.shuffle.spill helps a bit with large scale reduces
>
> Also it could be you're hitting:
> https://github.com/apache/incubator-spark/pull/180
>
> /Rafal
>
>
>
> Andrew Ash w
.memoryFraction (default is 0.66, try 0.5).
>
> Matei
>
> On Feb 5, 2014, at 10:29 PM, Andrew Ash wrote:
>
> // version 0.9.0
>
> Hi Spark users,
>
> My understanding of the MEMORY_AND_DISK_SER persistence level was that if
> an RDD could fit into memory then it w
// version 0.9.0
Hi Spark users,
My understanding of the MEMORY_AND_DISK_SER persistence level was that if
an RDD could fit into memory then it would be left there (same as
MEMORY_ONLY), and only if it was too big for memory would it spill to disk.
Here's how the docs describe it:
MEMORY_AND_DI
If you have multiple executors running on a single node then you might have
data that's on the same server but in different JVMs. Just on the same
server is NODE_LOCAL, but being in the same JVM is PROCESS_LOCAL.
Yes it was changed to be more specific than just preferred/non-preferred.
The new o
I'm observing this as well on 0.9.0, with several 10s of GB accumulating in
that directory but never being cleaned up. I think this has gotten more
pronounced in 0.9.0 as well with large reducers spilling to disk.
On Wed, Feb 5, 2014 at 3:46 PM, Mingyu Kim wrote:
> After creating a lot of Spar
When you look in the webui (port 8080) for the master does it list at least
one connected worker?
On Wed, Feb 5, 2014 at 7:19 AM, Soumya Simanta wrote:
> I'm running a Spark cluster. (Spark-0.9.0_SNAPSHOT).
>
> I connect to the Spark cluster from the spark-shell. I can see the Spark
> web UI on
I'm assuming you checked all the jars in SPARK_CLASSPATH to confirm that
parquet/org/codehaus/jackson/JsonGenerationException.class exists in one of
them?
On Wed, Feb 5, 2014 at 12:02 PM, Uri Laserson wrote:
> Has anyone tried this? I'd like to read a bunch of Avro GenericRecords
> from a Parq
Try depending on spark-core_2.10 rather than 2.10.3 -- the third digit was
dropped in the maven artifact and I hit this just yesterday as well.
Sent from my mobile phone
On Feb 5, 2014 10:41 AM, "Dana Tontea" wrote:
>Hi Matei,
>
> Firstly thank you a lot for answer.You are right I'm missing
> "hadoop fs" is almost certainly going to be better at copying these files
> than some memory-to-disk-to-memory serdes within Spark.
>
> --
> Christopher T. Nguyen
> Co-founder & CEO, Adatao <http://adatao.com>
> linkedin.com/in/ctnguyen
>
>
>
>
Hi Spark users,
I'm often using Spark for ETL type tasks, where the input is a large file
on-disk and the output is another large file on-disk. I've loaded
everything into HDFS, but still need to produce files out on the other side.
Right now I produce these processed files in a 2-step process:
The biggest difference I see is that Shark stores data in a Column-oriented
form a la C-Store and Vertica, whereas Spark keeps things in row-oriented
form. Chris pointed this out in the RDD[TablePartition] vs
RDD[Array[String]] comparison.
I'd be interested in hearing how TablePartition compares
Are you able to get a copy of the exception you refer to?
On Tue, Jan 28, 2014 at 2:26 AM, Ionized wrote:
> I noticed that running the following code results in the process hanging
> forever waiting for the Job to complete.
> It seems the exception never propagates to the caller.
>
> Should a b
Can you paste the exception you're seeing?
Sent from my mobile phone
On Jan 24, 2014 2:36 PM, "Kapil Malik" wrote:
> Hi all,
>
>
>
> Is it possible to create a Spark Context (i.e. the driver program) from a
> servlet deployed on some application server ?
>
> I am able to run spark Java driver s
#L549
On Fri, Jan 24, 2014 at 1:26 PM, Cheng Lian wrote:
> PairRDDFunctions.lookup is good enough in Spark, it's just that its time
> complexity is O(N). Of course, for RDDs equipped with a partitioner, N is
> the average size of a partition.
>
>
> On Sat, Jan 25, 2014 a
If you have a pair RDD (an RDD[A,B]) then you can use the .lookup() method
on it for faster access.
http://spark.incubator.apache.org/docs/latest/api/core/index.html#org.apache.spark.rdd.PairRDDFunctions
Spark's strength is running computations across a large set of data. If
you're trying to do
In Java you'd want to convert it to an entry set, which is a set of (key,
value) pairs from the hashmap. The closest I can see in scaladoc is the
.iterator method -- try that?
On Fri, Jan 24, 2014 at 12:56 PM, Manoj Samel wrote:
> Is there a way to create RDD over a hashmap ?
>
> If I have a ha
nt
>> codebase?
>>
>> On Jan 23, 2014, at 10:38 PM, Matei Zaharia
>> wrote:
>>
>> You’d have to add a filter after the cogroup too. Cogroup gives you (key,
>> (list of values in RDD 1, list in RDD 2)).
>>
>> Also one small thing, instead of setting
, which meant
> giving people quick access to all the operations that might be useful, and
> dealing with how we’ll implement those later. Over time it will be possible
> to divide things like RDD.scala into multiple traits if they become
> unwieldy.
>
> Matei
>
>
> On Jan
;
> On Thu, Jan 23, 2014 at 8:33 PM, Matei Zaharia wrote:
>
>> I’d be happy to see this added to the core API.
>>
>> Matei
>>
>> On Jan 23, 2014, at 5:39 PM, Andrew Ash wrote:
>>
>> Ah right of course -- perils of typing code without running it!
>>
keys land on valid values on the referenced table,
and the way I do that is checking to see what percentage of the references
actually land.
On Thu, Jan 23, 2014 at 6:36 PM, Evan R. Sparks wrote:
> Yup (well, with _._1 at the end!)
>
>
> On Thu, Jan 23, 2014 at 5:28 PM, Andrew Ash w
You're thinking like this?
A.map(v => (v,None)).join(B.map(v => (v,None))).map(_._2)
On Thu, Jan 23, 2014 at 6:26 PM, Evan R. Sparks wrote:
> You could map each to an RDD[(String,None)] and do a join.
>
>
> On Thu, Jan 23, 2014 at 5:18 PM, Andrew Ash wrote:
>
Hi spark users,
I recently wanted to calculate the set intersection of two RDDs of Strings.
I couldn't find a .intersection() method in the autocomplete or in the
Scala API docs, so used a little set theory to end up with this:
lazy val A = ...
lazy val B = ...
A.union(B).subtract(A.subtract(B))
Why can't you preprocess to filter out the bad rows? I often do this on
CSV files by testing if the raw line is "parseable" before splitting on ","
or similar. Just validate the line before attempting to apply BigDecimal
or anything like that.
Cheers,
Andrew
On Wed, Jan 22, 2014 at 9:04 PM, Ma
https://github.com/apache/incubator-spark/pull/489
On Tue, Jan 21, 2014 at 3:41 PM, Ognen Duzlevski <
og...@plainvanillagames.com> wrote:
> On Tue, Jan 21, 2014 at 10:37 PM, Andrew Ash wrote:
>
>> Documentation suggestion:
>>
>> Default number of tasks t
Documentation suggestion:
Default number of tasks to use *across the cluster* for distributed shuffle
operations (groupByKey, reduceByKey, etc) when
not set by user.
Ognen would that have clarified for you?
On Tue, Jan 21, 2014 at 3:35 PM, Matei Zaharia wrote:
> It’s just 4 over the whole clus
Also you will need to bounce the spark services from a new ssh session to
make the ulimit changes take effect (if you changed the value in
/etc/limits)
Sent from my mobile phone
On Jan 20, 2014 5:32 PM, "Jey Kottalam" wrote:
> Can you try ulimit -n to make sure the increased limit has taken effe
It sounds like the takeaway is that if you're using custom classes, you
need to make sure that their hashCode() and equals() methods are
value-based?
On Thu, Jan 16, 2014 at 12:08 PM, Patrick Wendell wrote:
> Thanks for following up and explaining this one! Definitely something
> other users mig
So for each (col2, col3) pair, you want the difference between the earliest
col1 value and the latest col1 value?
I'd suggest something like this:
val data = sc.textFile(...).map(l => l.split("\t"))
data.map(r => ((r(1), r(2)), r(0)) // produce an RDD of ((col2, col3), col1)
.groupByKey() //
I filed it and submitted the PR that Josh suggested:
https://spark-project.atlassian.net/browse/SPARK-1021
https://github.com/apache/incubator-spark/pull/379
On Wed, Jan 8, 2014 at 9:56 AM, Andrew Ash wrote:
> And at the moment we should use the atlassian.net Jira instance, not
I haven't seen that particular one before, but Shark only works with its
bundled version of Hive-0.9.0, not any other version. The reason is Shark
had to make some patches in Hive 0.9.0 so it's not vanilla 0.9.0, but
moving Shark to later versions of Hive takes some dev work that's not quite
lande
Hello,
Shark doesn't have a matching version to the recent Spark 0.8.1 release
yet. If you want to run Shark, you'll need to stick with Spark 0.8.0 for
the moment until Shark 0.8.1 is released. I'd guess dropping back on that
version would fix your problems.
Andrew
On Thu, Jan 9, 2014 at 1:23
My first thought on hearing that you're calling collect is that taking all
the data back to the driver is intensive on the network. Try checking the
basic systems stuff on the machines to get a sense of what's being heavily
used:
disk IO
CPU
network
Any kind of distributed system monitoring fram
Any exceptions you see in the worker machine's logs would be particularly
useful too.
On Wed, Jan 8, 2014 at 6:00 AM, Prashant Sharma wrote:
> Hi,
>
> Can you give a little more details about the problem apart from a few
> hints that would be great !. I would like to exactly what you did and how
To get shark on LZO files working (I have it up and running with CDH4.4.0)
you first need the hadoop-lzo jar on the classpath for shark (and spark).
Hadoop-lzo seems to require its native code component, unlike Hadoop which
can run non-native if it can't find native. So you'll need to add
hadoop-
ile official bugs in Jira, as long as it's not
> already there!
>
>
> On Tue, Jan 7, 2014 at 9:47 PM, Andrew Ash wrote:
>
>> Hi Josh,
>>
>> I just ran into this again myself and noticed that the source hasn't
>> changed since we discussed in
objects. We’d have to do the same in hadoopRDD and the various versions of
> that as well.
>
> Matei
>
> On Jan 8, 2014, at 12:38 AM, Andrew Ash wrote:
>
> Matei, do you mean something like A rather than B below?
>
> A) rdd.map(_.clone).cache
> B) rdd.cache
>
>
Matei, do you mean something like A rather than B below?
A) rdd.map(_.clone).cache
B) rdd.cache
I'd be happy to add documentation if there's a good place for it, but I'm
not sure there's an obvious place for it.
On Tue, Jan 7, 2014 at 9:35 PM, Matei Zaharia wrote:
> Yeah, unfortunately sequenc
the sortByKey that would make the initial partitioning sub-optimal.
>> Plus this way, it's a pain to use in the REPL.
>>
>> Cheers,
>>
>> Ryan
>>
>>
>> On Tue, Dec 10, 2013 at 7:06 AM, Andrew Ash wrote:
>>
>>> Since sortByKey() inv
n 7, 2014 at 9:20 AM, Aureliano Buendia wrote:
>
>
>
> On Tue, Jan 7, 2014 at 5:13 PM, Andrew Ash wrote:
>
>> If small-file is hosted in HDFS I think the default is one partition per
>> HDFS block. If it's in one block, which are 64MB each by default, that
>> might be
If small-file is hosted in HDFS I think the default is one partition per
HDFS block. If it's in one block, which are 64MB each by default, that
might be one partition.
Sent from my mobile phone
On Jan 7, 2014 8:46 AM, "Aureliano Buendia" wrote:
>
>
>
> On Thu, Jan 2,
licated on a lot of systems.
>> Regards
>> Mayur Rustagi
>> Ph: +919632149971
>> h <https://twitter.com/mayur_rustagi>ttp://www.sigmoidanalytics.com
>> https://twitter.com/mayur_rustagi
>>
>>
>>
>> On Thu, Jan 2, 2014 at 11:22 PM, Andrew A
I definitely think so. Network transfer is often a bottleneck for
distributed jobs, especially if you're using groupBys or re-keying things
often.
What network speed do you have between each HDFS node? 1GB?
On Fri, Jan 3, 2014 at 2:34 PM, Debasish Das wrote:
> Hi,
>
> I have HDFS and MapReduc
saveAsHadoopFile and saveAsNewAPIHadoopFile are on PairRDDFunctions which
uses some Scala magic to become available when you have an that's RDD[Key,
Value]
https://github.com/apache/incubator-spark/blob/master/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala#L648
Agreed, something
For hadoop properties I find the most reliable way to be to set them on a
Configuration object and use a method on SparkContext that accepts that
conf object.
>From working code:
import org.apache.hadoop.io.LongWritable
import org.apache.hadoop.io.Text
import org.apache.hadoop.conf.Configuration
2842][2^WP]]
> (43^1383836400^0[2^1380^1]6[2[5^Event][2^WP]]
>
>
> Not only it has reduced the no. of keys but also have modified it.
>
> groupBy operation only uses equals method of the Key class (to check the
> equality of the key), right?
>
>
> On Fri, Jan 3, 2014
Hi Archit,
A partition is a chunk of data about the size of an HDFS block, not that of
a single key. Because every partition is tracked individually and each is
processed in a task on one CPU core, having massive numbers of them causes
slowdowns in the scheduler and elsewhere in the system. Abou
In my spark-env.sh I append to the SPARK_CLASSPATH variable rather than
overriding it, because I want to support both adding a jar to all instances
of a shell (in spark-env.sh) and adding a jar to a single shell
instance (SPARK_CLASSPATH=/path/to/my.jar
/path/to/spark-shell)
That looks like this:
If you had RDD[[i, j, k], value] then you could reduce by j by essentially
mapping j into the key slot, doing the reduce, and then mapping it back:
rdd.map( ((i,j,k),v) => (j, (i, k, v)).reduce( ... ).map( (j,(i,k,v)) =>
((i,j,k),v))
It's not pretty, but I've had to use this pattern before too.
Do you have stacktraces or other errors for the workers' deaths?
This is a great case for using the Tanuki service wrapper that can be
configured to automatically restart JVMs that die.
Andrew
On Thu, Jan 2, 2014 at 4:38 PM, Debasish Das wrote:
> Hi,
>
> I have been running standalone spark cl
7;t. So, it seems like a bug. Should I file a Jira issue? I
> haven't done that yet for this project but would be happy to.
>
> Thanks,
> Philip
>
>
> On 1/2/2014 11:23 AM, Andrew Ash wrote:
>
> For testing, maybe try using .collect and doing the comparison bet
gt;
> On 1/2/2014 10:28 AM, Andrew Ash wrote:
>
> You want to write it to a local file on the machine? Try using
> "file:///path/to/target/mydir/" instead
>
> I'm not sure what behavior would be if you did this on a multi-machine
> cluster though -- you may
ur_rustagi
>
>
>
> On Thu, Jan 2, 2014 at 10:46 PM, Andrew Ash wrote:
>
>> Hi lihu,
>>
>> Maybe the data you're accessing is in in HDFS and only resides on 4 of
>> your 20 machines because it's only about 4 blocks (at default 64MB / block
>&
You want to write it to a local file on the machine? Try using
"file:///path/to/target/mydir/" instead
I'm not sure what behavior would be if you did this on a multi-machine
cluster though -- you may get a bit of data on each machine in that local
directory.
On Thu, Jan 2, 2014 at 12:22 PM, Phi
Hi lihu,
Maybe the data you're accessing is in in HDFS and only resides on 4 of your
20 machines because it's only about 4 blocks (at default 64MB / block
that's around a quarter GB). Where is your source data located and how is
it stored?
Andrew
On Thu, Jan 2, 2014 at 7:53 AM, lihu wrote:
>
Yes it will. This is called data locality and it works by matching the
hostname in Spark with the one in HDFS.
On Wed, Jan 1, 2014 at 2:40 AM, guxiaobo1982 wrote:
> Hi Andrew,
>
>
> Thanks for your reply, I have another question about using HDFS, when running
> hdfs and the standalone mode on
Hi Xiaobo,
I would recommend putting the files into an HDFS cluster on the same
machines instead if possible. If you're concerned about duplicating the
data, you can set the replication factor to 1 so you don't use more space
than before.
In my experience of Spark around 0.7.0 or so, when readin
Hi Aureliano,
It's very easy to get lines into (start byte number, line) using Hadoop's
TextInputFormat. See how SparkContext's textFile() method does it here:
https://github.com/apache/incubator-spark/blob/master/core/src/main/scala/org/apache/spark/SparkContext.scala#L291
What is the use case
due to java.io.IOException: Codec
> for file hdfs://hadoop00/tmp/ldpc_dec_top_245_to_275.vcd.sstv3.lzo
> not found, cannot run [duplicate 1]
>
> Rajeev Srivastava
> Silverline Design Inc
> 2118 Walsh ave, suite 204
> Santa Clara, CA, 95050
> cell : 408-409-0940
>
>
&g
ontents directly, or do you have to
> decompress them after reading them?
>
> Sent from my iPhone
>
> On Dec 24, 2013, at 12:03 AM, Andrew Ash wrote:
>
> Hi Rajeev,
>
> I'm not sure if you ever got it working, but I just got mine up and going.
> If you just us
; regards
> Rajeev
>
> Rajeev Srivastava
> Silverline Design Inc
> 2118 Walsh ave, suite 204
> Santa Clara, CA, 95050
> cell : 408-409-0940
>
>
> On Mon, Dec 16, 2013 at 11:24 AM, Andrew Ash wrote:
>
>> Hi Rajeev,
>>
>> It looks like you're using th
source destination
>>>
>>> Chain FORWARD (policy ACCEPT)
>>> target prot opt source destination
>>>
>>> Chain OUTPUT (policy ACCEPT)
>>> target prot opt source destination
>>>
>>
Hi Jie,
When you say firewall is closed does that mean ports are blocked between
the worker nodes? I believe workers start up on a random port and send
data directly between each other during shuffles. Your firewall may be
blocking those connections. Can you try with the firewall temporarily
di
Hi Rajeev,
It looks like you're using the com.hadoop.mapred.DeprecatedLzoTextInputFormat
input format above, while Stephen referred to com.hadoop.mapreduce.
LzoTextInputFormat
I think the way to use this in Spark would be to use the
SparkContext.hadoopFile() or SparkContext.newAPIHadoopFile() met
Hi Spark users,
I'm observing behavior where if a master node goes down for a restart, all
the worker JVMs die (in standalone cluster mode). In other cluster
computing setups with master-worker relationships (namely Hadoop), if a
worker can't connect to the master or its connection drops it retri
I'm interested in doing this too Rajeev. Did you make any progress?
On Mon, Dec 9, 2013 at 1:57 PM, Rajeev Srivastava
wrote:
> Hello experts,
> I would like to read a LZO splittable compressed file into spark.
> I have followed available material on the web on working with LZO
> compressed
Since sortByKey() invokes those right now, we should either a) change the
documentation to treat note that it kicks off actions or b) change the
method to execute those things lazily.
Personally I'd prefer b but don't know how difficult that would be.
On Tue, Dec 10, 2013 at 1:52 AM, Jason Lende
Also note that when you add parameters to the -cp flag on the JVM and want
to include multiple jars, the only way to do that is by including an entire
directory with "dir/*" -- you can't use "dir/*jar" or "dir/spark*jar" or
anything else like that.
http://stackoverflow.com/questions/219585/setting
LZO compression at a minimum, and using Parquet as a second step,
seems like the way to go though I haven't tried either personally yet.
Sent from my mobile phone
On Dec 8, 2013, at 16:54, Ankur Chauhan wrote:
> Hi all,
>
> Sorry for posting this again but I am interested in finding out what
>
> Hey Andrew, unfortunately I don’t know how easy this is. Maybe future
> versions of Akka have it. We can certainly ask them to do it in general but
> I imagine there are some use cases where they wanted this behavior.
>
> Matei
>
>
> On Dec 5, 2013, at 2:49 PM, Andrew Ash wr
Speaking of akka and host sensitivity... How much have you hacked on akka
to get it to support all of: myhost.mydomain.int, myhost, and 10.1.1.1?
It's kind of a pain to get the Spark URL to exactly match. I'm wondering
if there are usability gains that could be made here or if we're pretty
stuck
Hi Hao,
Where tasks go is influenced by where the data they operate on resides. If
the data is on one executor, it may make more sense to do all the
computation on that node rather than ship data across the network. How was
the data distributed across your cluster?
Andrew
On Mon, Dec 2, 2013
Hello,
I have a few questions about configuring memory usage on standalone
clusters. Can someone help me out?
1) The terms "slave" in ./bin/start-slaves.sh and "worker" in the docs seem
to be used interchangeably. Are they the same?
2) On a worker/slave, is there only one JVM running that has
How important is it that they're partitioned on hashcode() % 32 rather than
Spark's default partitioning?
In scala, you should be able to do this with
rdd.distinct.coalesce(32).mapPartitions(p => sorted(p))
I'm not sure what your end goal is here, but if it's just sort a bunch of
data and remove
At least from
http://stackoverflow.com/questions/817853/what-is-the-difference-between-serializable-and-externalizable-in-javait
looks like Externalizable is roughly an old-java version of
Serializable. Does that class implement both interfaces? Can you take
away the Externalizable interface if i
13-11-26 15:02:45,400 INFO [spark-akka.actor.default-dispatcher-4]
>> Starting task 2.0:0 as TID 104 on executor 0: (PROCESS_LOCAL)
>>
>> and the fact that the UI shows the RDD not partitioning across the
>> appropriate hbase region nodes. I was thinking this was
creation place as all the other variables that
> I have been using for months quite happily and that seem to impact Spark
> nicely. I have it set to Int.MaxValue.toString which I am guessing is large
> enough.
>
> It very occasionally will use all data local nodes, and sometimes a mi
Hi Erik,
I would guess that if you set spark.locality.wait to an absurdly large
value then you would have essentially that effect.
Maybe you aren't setting the system property before creating your Spark
context?
http://spark.incubator.apache.org/docs/latest/configuration.html
Andrew
On Tue, N
for all
> these things.
>
> Matei
>
> On Nov 25, 2013, at 4:59 PM, Andrew Ash wrote:
>
> How do you know Spark doesn't also use Kryo for shuffled files? Are there
> metrics or logs somewhere that make you believe it's normal Java
> serialization?
>
>
I believe this page has what you're looking for:
http://spark.incubator.apache.org/docs/latest/ec2-scripts.html
On Mon, Nov 25, 2013 at 4:57 PM, Walrus theCat wrote:
> Hi,
>
> I just updated my imports and tried to run my app using Spark 0.8, but it
> breaks. The AMI's spark-shell says it's 0.7
How do you know Spark doesn't also use Kryo for shuffled files? Are there
metrics or logs somewhere that make you believe it's normal Java
serialization?
On Mon, Nov 25, 2013 at 4:46 PM, Mayuresh Kunjir
wrote:
> This shows how to serialize user classes. I wanted Spark to serialize all
> shuffle
age. So if you set it super high you'll waste
> some memory temporarily allocating these buffers.
>
> The main issue with this IIRC was for fetching results, which we fixed
> in 0.8.0 to use a different communication library.
>
> - Patrick
>
> On Mon, Nov 25, 2013 at
There have been a number of threads on this list about needing to set
spark.akka.frameSize to something higher than the default. The issue seems
to come up most when one key in a groupByKey has particularly large amounts
of data.
What is the downside to setting this configuration parameter to the
Hi Spark list,
I'm looking to apply some iptables firewall rules to my spark cluster and
am not entirely sure what ports are required. I didn't see any specific
documentation of what ports Spark requires, so compiled this (incomplete)
list.
*From* *To* *Port (default)* *Purpose* admin machine ma
96 matches
Mail list logo