Wrong temp directory when compressing before sending text file to S3

2014-11-06 Thread Gary Malouf
We have some data that we are exporting from our HDFS cluster to S3 with
some help from Spark.  The final RDD command we run is:

csvData.saveAsTextFile(s3n://data/mess/2014/11/dump-oct-30-to-nov-5-gzip,
classOf[GzipCodec])


We have our 'spark.local.dir' set to our large ephemeral partition on
each slave (on EC2), but with compression on an intermediate format
seems to be written to /tmp/hadoop-root/s3.  Is this a bug in Spark or
are we missing a configuration property?


It's a problem for us because the root disks on EC2 xls are small (~ 5GB).


Parquet Migrations

2014-10-31 Thread Gary Malouf
Outside of what is discussed here
https://issues.apache.org/jira/browse/SPARK-3851 as a future solution, is
there any path for being able to modify a Parquet schema once some data has
been written?  This seems like the kind of thing that should make people
pause when considering whether or not to use Parquet+Spark...


Re: Parquet schema migrations

2014-10-24 Thread Gary Malouf
Hi Michael,

Does this affect people who use Hive for their metadata store as well?  I'm
wondering if the issue is as bad as I think it is - namely that if you
build up a year's worth of data, adding a field forces you to have to
migrate that entire year's data.

Gary

On Wed, Oct 8, 2014 at 5:08 PM, Cody Koeninger c...@koeninger.org wrote:

 On Wed, Oct 8, 2014 at 3:19 PM, Michael Armbrust mich...@databricks.com
 wrote:

 
  I was proposing you manually convert each different format into one
  unified format  (by adding literal nulls and such for missing columns)
 and
  then union these converted datasets.  It would be weird to have union all
  try and do this automatically.
 


 Sure, I was just musing on what an api for doing the merging without manual
 user input should look like / do.   I'll comment on the ticket, thanks for
 making it



Re: guava version conflicts

2014-09-22 Thread Gary Malouf
Hi Marcelo,

Interested to hear the approach to be taken.  Shading guava itself seems
extreme, but that might make sense.

Gary

On Sat, Sep 20, 2014 at 9:38 PM, Marcelo Vanzin van...@cloudera.com wrote:

 Hmm, looks like the hack to maintain backwards compatibility in the
 Java API didn't work that well. I'll take a closer look at this when I
 get to work on Monday.

 On Fri, Sep 19, 2014 at 10:30 PM, Cody Koeninger c...@koeninger.org
 wrote:
  After the recent spark project changes to guava shading, I'm seeing
 issues
  with the datastax spark cassandra connector (which depends on guava 15.0)
  and the datastax cql driver (which depends on guava 16.0.1)
 
  Building an assembly for a job (with spark marked as provided) that
  includes either guava 15.0 or 16.0.1, results in errors like the
 following:
 
  scala session.close
 
  scala s[14/09/20 04:56:35 ERROR Futures$CombinedFuture: input future
  failed.
  java.lang.IllegalAccessError: tried to access class
  org.spark-project.guava.common.base.Absent from class
  com.google.common.base.Optional
  at com.google.common.base.Optional.absent(Optional.java:79)
  at com.google.common.base.Optional.fromNullable(Optional.java:94)
  at
 
 com.google.common.util.concurrent.Futures$CombinedFuture.setOneValue(Futures.java:1608)
  at
 
 com.google.common.util.concurrent.Futures$CombinedFuture.access$400(Futures.java:1470)
  at
 
 com.google.common.util.concurrent.Futures$CombinedFuture$2.run(Futures.java:1548)
  at
 
 com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
  at
 
 com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
  at
 
 com.google.common.util.concurrent.ExecutionList.add(ExecutionList.java:101)
  at
 
 com.google.common.util.concurrent.AbstractFuture.addListener(AbstractFuture.java:170)
  at
 
 com.google.common.util.concurrent.Futures$CombinedFuture.init(Futures.java:1545)
  at
 
 com.google.common.util.concurrent.Futures$CombinedFuture.init(Futures.java:1491)
  at
  com.google.common.util.concurrent.Futures.listFuture(Futures.java:1640)
  at
  com.google.common.util.concurrent.Futures.allAsList(Futures.java:983)
  at
 
 com.datastax.driver.core.CloseFuture$Forwarding.init(CloseFuture.java:73)
  at
 
 com.datastax.driver.core.HostConnectionPool.closeAsync(HostConnectionPool.java:398)
  at
 
 com.datastax.driver.core.SessionManager.closeAsync(SessionManager.java:157)
  at
  com.datastax.driver.core.SessionManager.close(SessionManager.java:172)
  at
 
 com.datastax.spark.connector.cql.CassandraConnector$.com$datastax$spark$connector$cql$CassandraConnector$$destroySession(CassandraConnector.scala:180)
  at
 
 com.datastax.spark.connector.cql.CassandraConnector$$anonfun$5.apply(CassandraConnector.scala:151)
  at
 
 com.datastax.spark.connector.cql.CassandraConnector$$anonfun$5.apply(CassandraConnector.scala:151)
  at com.datastax.spark.connector.cql.RefCountedCache.com
 
 $datastax$spark$connector$cql$RefCountedCache$$releaseImmediately(RefCountedCache.scala:86)
  at
 
 com.datastax.spark.connector.cql.RefCountedCache$ReleaseTask.run(RefCountedCache.scala:26)
  at
 
 com.datastax.spark.connector.cql.RefCountedCache$$anonfun$com$datastax$spark$connector$cql$RefCountedCache$$processPendingReleases$2.apply(RefCountedCache.scala:150)
  at
 
 com.datastax.spark.connector.cql.RefCountedCache$$anonfun$com$datastax$spark$connector$cql$RefCountedCache$$processPendingReleases$2.apply(RefCountedCache.scala:147)
  at
 
 scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772)
  at scala.collection.Iterator$class.foreach(Iterator.scala:727)
  at
  scala.collection.concurrent.TrieMapIterator.foreach(TrieMap.scala:922)
  at
  scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
  at scala.collection.concurrent.TrieMap.foreach(TrieMap.scala:632)
  at
 
 scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
  at com.datastax.spark.connector.cql.RefCountedCache.com
 
 $datastax$spark$connector$cql$RefCountedCache$$processPendingReleases(RefCountedCache.scala:147)
  at
 
 com.datastax.spark.connector.cql.RefCountedCache$$anon$1.run(RefCountedCache.scala:157)
  at
  java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
  at
 
 java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
  at
 java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
  at
 
 java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
  at
 
 

Re: parquet predicate / projection pushdown into unionAll

2014-09-09 Thread Gary Malouf
I'm kind of surprised this was not run into before.  Do people not
segregate their data by day/week in the HDFS directory structure?


On Tue, Sep 9, 2014 at 2:08 PM, Michael Armbrust mich...@databricks.com
wrote:

 Thanks!

 On Tue, Sep 9, 2014 at 11:07 AM, Cody Koeninger c...@koeninger.org
 wrote:

  Opened
 
  https://issues.apache.org/jira/browse/SPARK-3462
 
  I'll take a look at ColumnPruning and see what I can do
 
  On Tue, Sep 9, 2014 at 12:46 PM, Michael Armbrust 
 mich...@databricks.com
  wrote:
 
  On Tue, Sep 9, 2014 at 10:17 AM, Cody Koeninger c...@koeninger.org
  wrote:
 
  Is there a reason in general not to push projections and predicates
 down
  into the individual ParquetTableScans in a union?
 
 
  This would be a great case to add to ColumnPruning.  Would be awesome if
  you could open a JIRA or even a PR :)
 
 
 



CoHadoop Papers

2014-08-26 Thread Gary Malouf
One of my colleagues has been questioning me as to why Spark/HDFS makes no
attempts to try to co-locate related data blocks.  He pointed to this
paper: http://www.vldb.org/pvldb/vol4/p575-eltabakh.pdf from 2011 on the
CoHadoop research and the performance improvements it yielded for
Map/Reduce jobs.

Would leveraging these ideas for writing data from Spark make sense/be
worthwhile?


Re: CoHadoop Papers

2014-08-26 Thread Gary Malouf
It appears support for this type of control over block placement is going
out in the next version of HDFS:
https://issues.apache.org/jira/browse/HDFS-2576


On Tue, Aug 26, 2014 at 7:43 AM, Gary Malouf malouf.g...@gmail.com wrote:

 One of my colleagues has been questioning me as to why Spark/HDFS makes no
 attempts to try to co-locate related data blocks.  He pointed to this
 paper: http://www.vldb.org/pvldb/vol4/p575-eltabakh.pdf from 2011 on the
 CoHadoop research and the performance improvements it yielded for
 Map/Reduce jobs.

 Would leveraging these ideas for writing data from Spark make sense/be
 worthwhile?





Re: CoHadoop Papers

2014-08-26 Thread Gary Malouf
Christopher, can you expand on the co-partitioning support?

We have a number of spark SQL tables (saved in parquet format) that all
could be considered to have a common hash key.  Our analytics team wants to
do frequent joins across these different data-sets based on this key.  It
makes sense that if the data for each key across 'tables' was co-located on
the same server, shuffles could be minimized and ultimately performance
could be much better.

From reading the HDFS issue I posted before, the way is being paved for
implementing this type of behavior though there are a lot of complications
to make it work I believe.


On Tue, Aug 26, 2014 at 10:40 AM, Christopher Nguyen c...@adatao.com wrote:

 Gary, do you mean Spark and HDFS separately, or Spark's use of HDFS?

 If the former, Spark does support copartitioning.

 If the latter, it's an HDFS scope that's outside of Spark. On that note,
 Hadoop does also make attempts to collocate data, e.g., rack awareness. I'm
 sure the paper makes useful contributions for its set of use cases.

 Sent while mobile. Pls excuse typos etc.
 On Aug 26, 2014 5:21 AM, Gary Malouf malouf.g...@gmail.com wrote:

 It appears support for this type of control over block placement is going
 out in the next version of HDFS:
 https://issues.apache.org/jira/browse/HDFS-2576


 On Tue, Aug 26, 2014 at 7:43 AM, Gary Malouf malouf.g...@gmail.com
 wrote:

  One of my colleagues has been questioning me as to why Spark/HDFS makes
 no
  attempts to try to co-locate related data blocks.  He pointed to this
  paper: http://www.vldb.org/pvldb/vol4/p575-eltabakh.pdf from 2011 on
 the
  CoHadoop research and the performance improvements it yielded for
  Map/Reduce jobs.
 
  Would leveraging these ideas for writing data from Spark make sense/be
  worthwhile?
 
 
 




Re: CoHadoop Papers

2014-08-26 Thread Gary Malouf
Hi Michael,

I think once that work is into HDFS, it will be great to expose this
functionality via Spark.  This is something worth pursuing because it could
grant orders of magnitude perf improvements in cases when people need to
join data.

The second item would be very interesting, could yield significant
performance boosts.

Best,

Gary


On Tue, Aug 26, 2014 at 6:50 PM, Michael Armbrust mich...@databricks.com
wrote:

 It seems like there are two things here:
  - Co-locating blocks with the same keys to avoid network transfer.
  - Leveraging partitioning information to avoid a shuffle when data is
 already partitioned correctly (even if those partitions aren't yet on the
 same machine).

 The former seems more complicated and probably requires the support from
 Hadoop you linked to.  However, the latter might be easier as there is
 already a framework for reasoning about partitioning and the need to
 shuffle in the Spark SQL planner.


 On Tue, Aug 26, 2014 at 8:37 AM, Gary Malouf malouf.g...@gmail.com
 wrote:

 Christopher, can you expand on the co-partitioning support?

 We have a number of spark SQL tables (saved in parquet format) that all
 could be considered to have a common hash key.  Our analytics team wants
 to
 do frequent joins across these different data-sets based on this key.  It
 makes sense that if the data for each key across 'tables' was co-located
 on
 the same server, shuffles could be minimized and ultimately performance
 could be much better.

 From reading the HDFS issue I posted before, the way is being paved for
 implementing this type of behavior though there are a lot of complications
 to make it work I believe.


 On Tue, Aug 26, 2014 at 10:40 AM, Christopher Nguyen c...@adatao.com
 wrote:

  Gary, do you mean Spark and HDFS separately, or Spark's use of HDFS?
 
  If the former, Spark does support copartitioning.
 
  If the latter, it's an HDFS scope that's outside of Spark. On that note,
  Hadoop does also make attempts to collocate data, e.g., rack awareness.
 I'm
  sure the paper makes useful contributions for its set of use cases.
 
  Sent while mobile. Pls excuse typos etc.
  On Aug 26, 2014 5:21 AM, Gary Malouf malouf.g...@gmail.com wrote:
 
  It appears support for this type of control over block placement is
 going
  out in the next version of HDFS:
  https://issues.apache.org/jira/browse/HDFS-2576
 
 
  On Tue, Aug 26, 2014 at 7:43 AM, Gary Malouf malouf.g...@gmail.com
  wrote:
 
   One of my colleagues has been questioning me as to why Spark/HDFS
 makes
  no
   attempts to try to co-locate related data blocks.  He pointed to this
   paper: http://www.vldb.org/pvldb/vol4/p575-eltabakh.pdf from 2011 on
  the
   CoHadoop research and the performance improvements it yielded for
   Map/Reduce jobs.
  
   Would leveraging these ideas for writing data from Spark make
 sense/be
   worthwhile?
  
  
  
 
 





Re: Mesos/Spark Deadlock

2014-08-25 Thread Gary Malouf
We have not tried the work-around because there are other bugs in there
that affected our set-up, though it seems it would help.


On Mon, Aug 25, 2014 at 12:54 AM, Timothy Chen tnac...@gmail.com wrote:

 +1 to have the work around in.

 I'll be investigating from the Mesos side too.

 Tim

 On Sun, Aug 24, 2014 at 9:52 PM, Matei Zaharia matei.zaha...@gmail.com
 wrote:
  Yeah, Mesos in coarse-grained mode probably wouldn't work here. It's too
 bad that this happens in fine-grained mode -- would be really good to fix.
 I'll see if we can get the workaround in
 https://github.com/apache/spark/pull/1860 into Spark 1.1. Incidentally
 have you tried that?
 
  Matei
 
  On August 23, 2014 at 4:30:27 PM, Gary Malouf (malouf.g...@gmail.com)
 wrote:
 
  Hi Matei,
 
  We have an analytics team that uses the cluster on a daily basis.  They
 use two types of 'run modes':
 
  1) For running actual queries, they set the spark.executor.memory to
 something between 4 and 8GB of RAM/worker.
 
  2) A shell that takes a minimal amount of memory on workers (128MB) for
 prototyping out a larger query.  This allows them to not take up RAM on the
 cluster when they do not really need it.
 
  We see the deadlocks when there are a few shells in either case.  From
 the usage patterns we have, coarse-grained mode would be a challenge as we
 have to constantly remind people to kill their shells as soon as their
 queries finish.
 
  Am I correct in viewing Mesos in coarse-grained mode as being similar to
 Spark Standalone's cpu allocation behavior?
 
 
 
 
  On Sat, Aug 23, 2014 at 7:16 PM, Matei Zaharia matei.zaha...@gmail.com
 wrote:
  Hey Gary, just as a workaround, note that you can use Mesos in
 coarse-grained mode by setting spark.mesos.coarse=true. Then it will hold
 onto CPUs for the duration of the job.
 
  Matei
 
  On August 23, 2014 at 7:57:30 AM, Gary Malouf (malouf.g...@gmail.com)
 wrote:
 
  I just wanted to bring up a significant Mesos/Spark issue that makes the
  combo difficult to use for teams larger than 4-5 people. It's covered in
  https://issues.apache.org/jira/browse/MESOS-1688. My understanding is
 that
  Spark's use of executors in fine-grained mode is a very different
 behavior
  than many of the other common frameworks for Mesos.
 



Mesos/Spark Deadlock

2014-08-23 Thread Gary Malouf
I just wanted to bring up a significant Mesos/Spark issue that makes the
combo difficult to use for teams larger than 4-5 people.  It's covered in
https://issues.apache.org/jira/browse/MESOS-1688.  My understanding is that
Spark's use of executors in fine-grained mode is a very different behavior
than many of the other common frameworks for Mesos.


Re: Mesos/Spark Deadlock

2014-08-23 Thread Gary Malouf
Hi Matei,

We have an analytics team that uses the cluster on a daily basis.  They use
two types of 'run modes':

1) For running actual queries, they set the spark.executor.memory to
something between 4 and 8GB of RAM/worker.

2) A shell that takes a minimal amount of memory on workers (128MB) for
prototyping out a larger query.  This allows them to not take up RAM on the
cluster when they do not really need it.

We see the deadlocks when there are a few shells in either case.  From the
usage patterns we have, coarse-grained mode would be a challenge as we have
to constantly remind people to kill their shells as soon as their queries
finish.

Am I correct in viewing Mesos in coarse-grained mode as being similar to
Spark Standalone's cpu allocation behavior?




On Sat, Aug 23, 2014 at 7:16 PM, Matei Zaharia matei.zaha...@gmail.com
wrote:

 Hey Gary, just as a workaround, note that you can use Mesos in
 coarse-grained mode by setting spark.mesos.coarse=true. Then it will hold
 onto CPUs for the duration of the job.

 Matei

 On August 23, 2014 at 7:57:30 AM, Gary Malouf (malouf.g...@gmail.com)
 wrote:

 I just wanted to bring up a significant Mesos/Spark issue that makes the
 combo difficult to use for teams larger than 4-5 people. It's covered in
 https://issues.apache.org/jira/browse/MESOS-1688. My understanding is
 that
 Spark's use of executors in fine-grained mode is a very different behavior
 than many of the other common frameworks for Mesos.




Re: [SPARK-3050] Spark program running with 1.0.2 jar cannot run against a 1.0.1 cluster

2014-08-14 Thread Gary Malouf
To be clear, is it 'compiled' against 1.0.2 or it packaged with it?


On Thu, Aug 14, 2014 at 6:39 PM, Mingyu Kim m...@palantir.com wrote:

 I ran a really simple code that runs with Spark 1.0.2 jar and connects to
 a Spark 1.0.1 cluster, but it fails with java.io.InvalidClassException. I
 filed the bug at https://issues.apache.org/jira/browse/SPARK-3050.

 I assumed the minor and patch releases shouldn’t break compatibility. Is
 that correct?

 Thanks,
 Mingyu



Re: replacement for SPARK_JAVA_OPTS

2014-08-07 Thread Gary Malouf
Can this be cherry-picked for 1.1 if everything works out?  In my opinion,
it could be qualified as a bug fix.


On Thu, Aug 7, 2014 at 5:47 PM, Marcelo Vanzin van...@cloudera.com wrote:

 Andrew has been working on a fix:
 https://github.com/apache/spark/pull/1770

 On Thu, Aug 7, 2014 at 2:35 PM, Cody Koeninger c...@koeninger.org wrote:
  Just wanted to check in on this, see if I should file a bug report
  regarding the mesos argument propagation.
 
 
  On Thu, Jul 31, 2014 at 8:35 AM, Cody Koeninger c...@koeninger.org
 wrote:
 
  1. I've tried with and without escaping equals sign, it doesn't affect
 the
  results.
 
  2. Yeah, exporting SPARK_SUBMIT_OPTS from spark-env.sh works for getting
  system properties set in the local shell (although not for executors).
 
  3. We're using the default fine-grained mesos mode, not setting
  spark.mesos.coarse, so it doesn't seem immediately related to that
 ticket.
  Should I file a bug report?
 
 
  On Thu, Jul 31, 2014 at 1:33 AM, Patrick Wendell pwend...@gmail.com
  wrote:
 
  The third issue may be related to this:
  https://issues.apache.org/jira/browse/SPARK-2022
 
  We can take a look at this during the bug fix period for the 1.1
  release next week. If we come up with a fix we can backport it into
  the 1.0 branch also.
 
  On Wed, Jul 30, 2014 at 11:31 PM, Patrick Wendell pwend...@gmail.com
  wrote:
   Thanks for digging around here. I think there are a few distinct
 issues.
  
   1. Properties containing the '=' character need to be escaped.
   I was able to load properties fine as long as I escape the '='
   character. But maybe we should document this:
  
   == spark-defaults.conf ==
   spark.foo a\=B
   == shell ==
   scala sc.getConf.get(spark.foo)
   res2: String = a=B
  
   2. spark.driver.extraJavaOptions, when set in the properties file,
   don't affect the driver when running in client mode (always the case
   for mesos). We should probably document this. In this case you need
 to
   either use --driver-java-options or set SPARK_SUBMIT_OPTS.
  
   3. Arguments aren't propagated on Mesos (this might be because of the
   other issues, or a separate bug).
  
   - Patrick
  
   On Wed, Jul 30, 2014 at 3:10 PM, Cody Koeninger c...@koeninger.org
  wrote:
   In addition, spark.executor.extraJavaOptions does not seem to behave
  as I
   would expect; java arguments don't seem to be propagated to
 executors.
  
  
   $ cat conf/spark-defaults.conf
  
   spark.master
  
 
 mesos://zk://etl-01.mxstg:2181,etl-02.mxstg:2181,etl-03.mxstg:2181/masters
   spark.executor.extraJavaOptions -Dfoo.bar.baz=23
   spark.driver.extraJavaOptions -Dfoo.bar.baz=23
  
  
   $ ./bin/spark-shell
  
   scala sc.getConf.get(spark.executor.extraJavaOptions)
   res0: String = -Dfoo.bar.baz=23
  
   scala sc.parallelize(1 to 100).map{ i = (
|  java.net.InetAddress.getLocalHost.getHostName,
|  System.getProperty(foo.bar.baz)
| )}.collect
  
   res1: Array[(String, String)] = Array((dn-01.mxstg,null),
   (dn-01.mxstg,null), (dn-01.mxstg,null), (dn-01.mxstg,null),
   (dn-01.mxstg,null), (dn-01.mxstg,null), (dn-01.mxstg,null),
   (dn-01.mxstg,null), (dn-01.mxstg,null), (dn-01.mxstg,null),
   (dn-01.mxstg,null), (dn-01.mxstg,null), (dn-02.mxstg,null),
   (dn-02.mxstg,null), ...
  
  
  
   Note that this is a mesos deployment, although I wouldn't expect
 that
  to
   affect the availability of spark.driver.extraJavaOptions in a local
  spark
   shell.
  
  
   On Wed, Jul 30, 2014 at 4:18 PM, Cody Koeninger c...@koeninger.org
 
  wrote:
  
   Either whitespace or equals sign are valid properties file formats.
   Here's an example:
  
   $ cat conf/spark-defaults.conf
   spark.driver.extraJavaOptions -Dfoo.bar.baz=23
  
   $ ./bin/spark-shell -v
   Using properties file: /opt/spark/conf/spark-defaults.conf
   Adding default property:
  spark.driver.extraJavaOptions=-Dfoo.bar.baz=23
  
  
   scala  System.getProperty(foo.bar.baz)
   res0: String = null
  
  
   If you add double quotes, the resulting string value will have
 double
   quotes.
  
  
   $ cat conf/spark-defaults.conf
   spark.driver.extraJavaOptions -Dfoo.bar.baz=23
  
   $ ./bin/spark-shell -v
   Using properties file: /opt/spark/conf/spark-defaults.conf
   Adding default property:
  spark.driver.extraJavaOptions=-Dfoo.bar.baz=23
  
   scala  System.getProperty(foo.bar.baz)
   res0: String = null
  
  
   Neither one of those affects the issue; the underlying problem in
 my
  case
   seems to be that bin/spark-class uses the SPARK_SUBMIT_OPTS and
   SPARK_JAVA_OPTS environment variables, but nothing parses
   spark-defaults.conf before the java process is started.
  
   Here's an example of the process running when only
  spark-defaults.conf is
   being used:
  
   $ ps -ef | grep spark
  
   514   5182  2058  0 21:05 pts/200:00:00 bash
  ./bin/spark-shell -v
  
   514   5189  5182  4 21:05 pts/200:00:22
  /usr/local/java/bin/java
   -cp
  
 
 

Kryo Issue on Spark 1.0.1, Mesos 0.18.2

2014-07-25 Thread Gary Malouf
After upgrading to Spark 1.0.1 from 0.9.1 everything seemed to be going
well.  Looking at the Mesos slave logs, I noticed:

ERROR KryoSerializer: Failed to run spark.kryo.registrator
java.lang.ClassNotFoundException:
com/mediacrossing/verrazano/kryo/MxDataRegistrator

My spark-env.sh has the following when I run the Spark Shell:

export MESOS_NATIVE_LIBRARY=/usr/local/lib/libmesos.so

export MASTER=mesos://zk://n-01:2181,n-02:2181,n-03:2181/masters

export ADD_JARS=/opt/spark/mx-lib/verrazano-assembly.jar


# -XX:+UseCompressedOops must be disabled to use more than 32GB RAM

SPARK_JAVA_OPTS=-Xss2m -XX:+UseCompressedOops
-Dspark.local.dir=/opt/mesos-tmp -Dspark.executor.memory=4g
 -Dspark.serializer=org.apache.spark.serializer.KryoSerializer
-Dspark.kryo.registrator=com.mediacrossing.verrazano.kryo.MxDataRegistrator
-Dspark.kryoserializer.buffer.mb=16 -Dspark.akka.askTimeout=30


I was able to verify that our custom jar was being copied to each worker,
but for some reason it is not finding my registrator class.  Is anyone else
struggling with Kryo on 1.0.x branch?


Re: Reproducible deadlock in 1.0.1, possibly related to Spark-1097

2014-07-14 Thread Gary Malouf
We use the Hadoop configuration inside of our code executing on Spark as we
need to list out files in the path.  Maybe that is why it is exposed for us.


On Mon, Jul 14, 2014 at 6:57 PM, Patrick Wendell pwend...@gmail.com wrote:

 Hey Nishkam,

 Aaron's fix should prevent two concurrent accesses to getJobConf (and
 the Hadoop code therein). But if there is code elsewhere that tries to
 mutate the configuration, then I could see how we might still have the
 ConcurrentModificationException.

 I looked at your patch for HADOOP-10456 and the only example you give
 is of the data being accessed inside of getJobConf. Is it accessed
 somewhere else too from Spark that you are aware of?

 https://issues.apache.org/jira/browse/HADOOP-10456

 - Patrick

 On Mon, Jul 14, 2014 at 3:28 PM, Nishkam Ravi nr...@cloudera.com wrote:
  Hi Aaron, I'm not sure if synchronizing on an arbitrary lock object would
  help. I suspect we will start seeing the ConcurrentModificationException
  again. The right fix has gone into Hadoop through 10456. Unfortunately, I
  don't have any bright ideas on how to synchronize this at the Spark level
  without the risk of deadlocks.
 
 
  On Mon, Jul 14, 2014 at 3:07 PM, Aaron Davidson ilike...@gmail.com
 wrote:
 
  The full jstack would still be useful, but our current working theory is
  that this is due to the fact that Configuration#loadDefaults goes
 through
  every Configuration object that was ever created (via
  Configuration.REGISTRY) and locks it, thus introducing a dependency from
  new Configuration to old, otherwise unrelated, Configuration objects
 that
  our locking did not anticipate.
 
  I have created https://github.com/apache/spark/pull/1409 to hopefully
 fix
  this bug.
 
 
  On Mon, Jul 14, 2014 at 2:44 PM, Patrick Wendell pwend...@gmail.com
  wrote:
 
   Hey Cody,
  
   This Jstack seems truncated, would you mind giving the entire stack
   trace? For the second thread, for instance, we can't see where the
   lock is being acquired.
  
   - Patrick
  
   On Mon, Jul 14, 2014 at 1:42 PM, Cody Koeninger
   cody.koenin...@mediacrossing.com wrote:
Hi all, just wanted to give a heads up that we're seeing a
 reproducible
deadlock with spark 1.0.1 with 2.3.0-mr1-cdh5.0.2
   
If jira is a better place for this, apologies in advance - figured
   talking
about it on the mailing list was friendlier than randomly
 (re)opening
   jira
tickets.
   
I know Gary had mentioned some issues with 1.0.1 on the mailing
 list,
   once
we got a thread dump I wanted to follow up.
   
The thread dump shows the deadlock occurs in the synchronized block
 of
   code
that was changed in HadoopRDD.scala, for the Spark-1097 issue
   
Relevant portions of the thread dump are summarized below, we can
  provide
the whole dump if it's useful.
   
Found one Java-level deadlock:
=
Executor task launch worker-1:
  waiting to lock monitor 0x7f250400c520 (object
  0xfae7dc30,
   a
org.apache.hadoop.co
nf.Configuration),
  which is held by Executor task launch worker-0
Executor task launch worker-0:
  waiting to lock monitor 0x7f2520495620 (object
  0xfaeb4fc8,
   a
java.lang.Class),
  which is held by Executor task launch worker-1
   
   
Executor task launch worker-1:
at
   
  
 
 org.apache.hadoop.conf.Configuration.reloadConfiguration(Configuration.java:791)
- waiting to lock 0xfae7dc30 (a
org.apache.hadoop.conf.Configuration)
at
   
  
 
 org.apache.hadoop.conf.Configuration.addDefaultResource(Configuration.java:690)
- locked 0xfaca6ff8 (a java.lang.Class for
org.apache.hadoop.conf.Configurati
on)
at
   
  
 
 org.apache.hadoop.hdfs.HdfsConfiguration.clinit(HdfsConfiguration.java:34)
at
   
  
 
 org.apache.hadoop.hdfs.DistributedFileSystem.clinit(DistributedFileSystem.java:110
)
at
  sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
Method)
at
   
  
 
 sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.
java:57)
at
  sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native
Method)
at
   
  
 
 sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.
java:57)
at
   
  
 
 sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAcces
sorImpl.java:45)
at
   java.lang.reflect.Constructor.newInstance(Constructor.java:525)
at java.lang.Class.newInstance0(Class.java:374)
at java.lang.Class.newInstance(Class.java:327)
at
   java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:373)
at java.util.ServiceLoader$1.next(ServiceLoader.java:445)
at
   
 org.apache.hadoop.fs.FileSystem.loadFileSystems(FileSystem.java:2364)
 

Re: Reproducible deadlock in 1.0.1, possibly related to Spark-1097

2014-07-14 Thread Gary Malouf
We'll try to run a build tomorrow AM.


On Mon, Jul 14, 2014 at 7:22 PM, Patrick Wendell pwend...@gmail.com wrote:

 Andrew and Gary,

 Would you guys be able to test
 https://github.com/apache/spark/pull/1409/files and see if it solves
 your problem?

 - Patrick

 On Mon, Jul 14, 2014 at 4:18 PM, Andrew Ash and...@andrewash.com wrote:
  I observed a deadlock here when using the AvroInputFormat as well. The
  short of the issue is that there's one configuration object per JVM, but
  multiple threads, one for each task. If each thread attempts to add a
  configuration option to the Configuration object at once you get issues
  because HashMap isn't thread safe.
 
  More details to come tonight. Thanks!
  On Jul 14, 2014 4:11 PM, Nishkam Ravi nr...@cloudera.com wrote:
 
  HI Patrick, I'm not aware of another place where the access happens, but
  it's possible that it does. The original fix synchronized on the
  broadcastConf object and someone reported the same exception.
 
 
  On Mon, Jul 14, 2014 at 3:57 PM, Patrick Wendell pwend...@gmail.com
  wrote:
 
   Hey Nishkam,
  
   Aaron's fix should prevent two concurrent accesses to getJobConf (and
   the Hadoop code therein). But if there is code elsewhere that tries to
   mutate the configuration, then I could see how we might still have the
   ConcurrentModificationException.
  
   I looked at your patch for HADOOP-10456 and the only example you give
   is of the data being accessed inside of getJobConf. Is it accessed
   somewhere else too from Spark that you are aware of?
  
   https://issues.apache.org/jira/browse/HADOOP-10456
  
   - Patrick
  
   On Mon, Jul 14, 2014 at 3:28 PM, Nishkam Ravi nr...@cloudera.com
  wrote:
Hi Aaron, I'm not sure if synchronizing on an arbitrary lock object
  would
help. I suspect we will start seeing the
  ConcurrentModificationException
again. The right fix has gone into Hadoop through 10456.
  Unfortunately, I
don't have any bright ideas on how to synchronize this at the Spark
  level
without the risk of deadlocks.
   
   
On Mon, Jul 14, 2014 at 3:07 PM, Aaron Davidson ilike...@gmail.com
 
   wrote:
   
The full jstack would still be useful, but our current working
 theory
  is
that this is due to the fact that Configuration#loadDefaults goes
   through
every Configuration object that was ever created (via
Configuration.REGISTRY) and locks it, thus introducing a dependency
  from
new Configuration to old, otherwise unrelated, Configuration
 objects
   that
our locking did not anticipate.
   
I have created https://github.com/apache/spark/pull/1409 to
 hopefully
   fix
this bug.
   
   
On Mon, Jul 14, 2014 at 2:44 PM, Patrick Wendell 
 pwend...@gmail.com
wrote:
   
 Hey Cody,

 This Jstack seems truncated, would you mind giving the entire
 stack
 trace? For the second thread, for instance, we can't see where
 the
 lock is being acquired.

 - Patrick

 On Mon, Jul 14, 2014 at 1:42 PM, Cody Koeninger
 cody.koenin...@mediacrossing.com wrote:
  Hi all, just wanted to give a heads up that we're seeing a
   reproducible
  deadlock with spark 1.0.1 with 2.3.0-mr1-cdh5.0.2
 
  If jira is a better place for this, apologies in advance -
 figured
 talking
  about it on the mailing list was friendlier than randomly
   (re)opening
 jira
  tickets.
 
  I know Gary had mentioned some issues with 1.0.1 on the mailing
   list,
 once
  we got a thread dump I wanted to follow up.
 
  The thread dump shows the deadlock occurs in the synchronized
  block
   of
 code
  that was changed in HadoopRDD.scala, for the Spark-1097 issue
 
  Relevant portions of the thread dump are summarized below, we
 can
provide
  the whole dump if it's useful.
 
  Found one Java-level deadlock:
  =
  Executor task launch worker-1:
waiting to lock monitor 0x7f250400c520 (object
0xfae7dc30,
 a
  org.apache.hadoop.co
  nf.Configuration),
which is held by Executor task launch worker-0
  Executor task launch worker-0:
waiting to lock monitor 0x7f2520495620 (object
0xfaeb4fc8,
 a
  java.lang.Class),
which is held by Executor task launch worker-1
 
 
  Executor task launch worker-1:
  at
 

   
  
 
 org.apache.hadoop.conf.Configuration.reloadConfiguration(Configuration.java:791)
  - waiting to lock 0xfae7dc30 (a
  org.apache.hadoop.conf.Configuration)
  at
 

   
  
 
 org.apache.hadoop.conf.Configuration.addDefaultResource(Configuration.java:690)
  - locked 0xfaca6ff8 (a java.lang.Class for
  org.apache.hadoop.conf.Configurati
  on)
  at
 

   
  
 
 org.apache.hadoop.hdfs.HdfsConfiguration.clinit(HdfsConfiguration.java:34)

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

2014-07-11 Thread Gary Malouf
Hi Matei,

We have not had time to re-deploy the rc today, but one thing that jumps
out is the shrinking of the default akka frame size from 10MB to around
128KB by default.  That is my first suspicion for our issue - could imagine
that biting others as well.

I'll try to re-test that today - either way, understand moving forward at
this point.

Gary


On Fri, Jul 11, 2014 at 12:08 PM, Matei Zaharia matei.zaha...@gmail.com
wrote:

 Unless you can diagnose the problem quickly, Gary, I think we need to go
 ahead with this release as is. This release didn't touch the Mesos support
 as far as I know, so the problem might be a nondeterministic issue with
 your application. But on the other hand the release does fix some critical
 bugs that affect all users. We can always do 1.0.2 later if we discover a
 problem.

 Matei

 On Jul 10, 2014, at 9:40 PM, Patrick Wendell pwend...@gmail.com wrote:

  Hey Gary,
 
  The vote technically doesn't close until I send the vote summary
  e-mail, but I was planning to close and package this tonight. It's too
  bad if there is a regression, it might be worth holding the release
  but it really requires narrowing down the issue to get more
  information about the scope and severity. Could you fork another
  thread for this?
 
  - Patrick
 
  On Thu, Jul 10, 2014 at 6:28 PM, Gary Malouf malouf.g...@gmail.com
 wrote:
  -1 I honestly do not know the voting rules for the Spark community, so
  please excuse me if I am out of line or if Mesos compatibility is not a
  concern at this point.
 
  We just tried to run this version built against 2.3.0-cdh5.0.2 on mesos
  0.18.2.  All of our jobs with data above a few gigabytes hung
 indefinitely.
  Downgrading back to the 1.0.0 stable release of Spark built the same way
  worked for us.
 
 
  On Mon, Jul 7, 2014 at 5:17 PM, Tom Graves tgraves...@yahoo.com.invalid
 
  wrote:
 
  +1. Ran some Spark on yarn jobs on a hadoop 2.4 cluster with
  authentication on.
 
  Tom
 
 
  On Friday, July 4, 2014 2:39 PM, Patrick Wendell pwend...@gmail.com
  wrote:
 
 
 
  Please vote on releasing the following candidate as Apache Spark
 version
  1.0.1!
 
  The tag to be voted on is v1.0.1-rc1 (commit 7d1043c):
 
 
 https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=7d1043c99303b87aef8ee19873629c2bfba4cc78
 
  The release files, including signatures, digests, etc. can be found at:
  http://people.apache.org/~pwendell/spark-1.0.1-rc2/
 
  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-1021/
 
  The documentation corresponding to this release can be found at:
  http://people.apache.org/~pwendell/spark-1.0.1-rc2-docs/
 
  Please vote on releasing this package as Apache Spark 1.0.1!
 
  The vote is open until Monday, July 07, at 20:45 UTC and passes if
  a majority of at least 3 +1 PMC votes are cast.
 
  [ ] +1 Release this package as Apache Spark 1.0.1
  [ ] -1 Do not release this package because ...
 
  To learn more about Apache Spark, please see
  http://spark.apache.org/
 
  === Differences from RC1 ===
  This release includes only one blocking patch from rc1:
  https://github.com/apache/spark/pull/1255
 
  There are also smaller fixes which came in over the last week.
 
  === About this release ===
  This release fixes a few high-priority bugs in 1.0 and has a variety
  of smaller fixes. The full list is here: http://s.apache.org/b45. Some
  of the more visible patches are:
 
  SPARK-2043: ExternalAppendOnlyMap doesn't always find matching keys
  SPARK-2156 and SPARK-1112: Issues with jobs hanging due to akka frame
 size.
  SPARK-1790: Support r3 instance types on EC2.
 
  This is the first maintenance release on the 1.0 line. We plan to make
  additional maintenance releases as new fixes come in.
 




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

2014-07-10 Thread Gary Malouf
-1 I honestly do not know the voting rules for the Spark community, so
please excuse me if I am out of line or if Mesos compatibility is not a
concern at this point.

We just tried to run this version built against 2.3.0-cdh5.0.2 on mesos
0.18.2.  All of our jobs with data above a few gigabytes hung indefinitely.
 Downgrading back to the 1.0.0 stable release of Spark built the same way
worked for us.


On Mon, Jul 7, 2014 at 5:17 PM, Tom Graves tgraves...@yahoo.com.invalid
wrote:

 +1. Ran some Spark on yarn jobs on a hadoop 2.4 cluster with
 authentication on.

 Tom


 On Friday, July 4, 2014 2:39 PM, Patrick Wendell pwend...@gmail.com
 wrote:



 Please vote on releasing the following candidate as Apache Spark version
 1.0.1!

 The tag to be voted on is v1.0.1-rc1 (commit 7d1043c):

 https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=7d1043c99303b87aef8ee19873629c2bfba4cc78

 The release files, including signatures, digests, etc. can be found at:
 http://people.apache.org/~pwendell/spark-1.0.1-rc2/

 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-1021/

 The documentation corresponding to this release can be found at:
 http://people.apache.org/~pwendell/spark-1.0.1-rc2-docs/

 Please vote on releasing this package as Apache Spark 1.0.1!

 The vote is open until Monday, July 07, at 20:45 UTC and passes if
 a majority of at least 3 +1 PMC votes are cast.

 [ ] +1 Release this package as Apache Spark 1.0.1
 [ ] -1 Do not release this package because ...

 To learn more about Apache Spark, please see
 http://spark.apache.org/

 === Differences from RC1 ===
 This release includes only one blocking patch from rc1:
 https://github.com/apache/spark/pull/1255

 There are also smaller fixes which came in over the last week.

 === About this release ===
 This release fixes a few high-priority bugs in 1.0 and has a variety
 of smaller fixes. The full list is here: http://s.apache.org/b45. Some
 of the more visible patches are:

 SPARK-2043: ExternalAppendOnlyMap doesn't always find matching keys
 SPARK-2156 and SPARK-1112: Issues with jobs hanging due to akka frame size.
 SPARK-1790: Support r3 instance types on EC2.

 This is the first maintenance release on the 1.0 line. We plan to make
 additional maintenance releases as new fixes come in.



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

2014-07-10 Thread Gary Malouf
Just realized the deadline was Monday, my apologies.  The issue
nevertheless stands.


On Thu, Jul 10, 2014 at 9:28 PM, Gary Malouf malouf.g...@gmail.com wrote:

 -1 I honestly do not know the voting rules for the Spark community, so
 please excuse me if I am out of line or if Mesos compatibility is not a
 concern at this point.

 We just tried to run this version built against 2.3.0-cdh5.0.2 on mesos
 0.18.2.  All of our jobs with data above a few gigabytes hung indefinitely.
  Downgrading back to the 1.0.0 stable release of Spark built the same way
 worked for us.


 On Mon, Jul 7, 2014 at 5:17 PM, Tom Graves tgraves...@yahoo.com.invalid
 wrote:

 +1. Ran some Spark on yarn jobs on a hadoop 2.4 cluster with
 authentication on.

 Tom


 On Friday, July 4, 2014 2:39 PM, Patrick Wendell pwend...@gmail.com
 wrote:



 Please vote on releasing the following candidate as Apache Spark version
 1.0.1!

 The tag to be voted on is v1.0.1-rc1 (commit 7d1043c):

 https://git-wip-us.apache.org/repos/asf?p=spark.git;a=commit;h=7d1043c99303b87aef8ee19873629c2bfba4cc78

 The release files, including signatures, digests, etc. can be found at:
 http://people.apache.org/~pwendell/spark-1.0.1-rc2/

 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-1021/

 The documentation corresponding to this release can be found at:
 http://people.apache.org/~pwendell/spark-1.0.1-rc2-docs/

 Please vote on releasing this package as Apache Spark 1.0.1!

 The vote is open until Monday, July 07, at 20:45 UTC and passes if
 a majority of at least 3 +1 PMC votes are cast.

 [ ] +1 Release this package as Apache Spark 1.0.1
 [ ] -1 Do not release this package because ...

 To learn more about Apache Spark, please see
 http://spark.apache.org/

 === Differences from RC1 ===
 This release includes only one blocking patch from rc1:
 https://github.com/apache/spark/pull/1255

 There are also smaller fixes which came in over the last week.

 === About this release ===
 This release fixes a few high-priority bugs in 1.0 and has a variety
 of smaller fixes. The full list is here: http://s.apache.org/b45. Some
 of the more visible patches are:

 SPARK-2043: ExternalAppendOnlyMap doesn't always find matching keys
 SPARK-2156 and SPARK-1112: Issues with jobs hanging due to akka frame
 size.
 SPARK-1790: Support r3 instance types on EC2.

 This is the first maintenance release on the 1.0 line. We plan to make
 additional maintenance releases as new fixes come in.





Re: Spark on Scala 2.11

2014-05-10 Thread Gary Malouf
Considering the team just bumped to 2.10 in 0.9, I would be surprised if
this is a near term priority.


On Thu, May 8, 2014 at 9:33 PM, Anand Avati av...@gluster.org wrote:

 Is there an ongoing effort (or intent) to support Spark on Scala 2.11?
 Approximate timeline?

 Thanks