Re: Multiple sessions in one application?

2018-12-21 Thread Mark Hamstra
On the contrary, it is a common occurrence in a Spark Jobserver style of application with multiple users. On Thu, Dec 20, 2018 at 6:09 PM Jiaan Geng wrote: > This scene is rare. > When you provide a web server for spark. maybe you need it. > > > > -- > Sent from:

Re: How to track batch jobs in spark ?

2018-12-05 Thread Mark Hamstra
That will kill an entire Spark application, not a batch Job. On Wed, Dec 5, 2018 at 3:07 PM Priya Matpadi wrote: > if you are deploying your spark application on YARN cluster, > 1. ssh into master node > 2. List the currently running application and retreive the application_id > yarn

Re: Scala: The Util is not accessible in def main

2018-11-11 Thread Mark Hamstra
It is intentionally not accessible in your code since Utils is internal Spark code, not part of the public API. Changing Spark to make that private code public would be inviting trouble, or at least future headaches. If you don't already know how to build and maintain your own custom fork of Spark

Re: Custom SparkListener

2018-09-20 Thread Mark Hamstra
What do you mean? Spark Jobs don't have names. On Thu, Sep 20, 2018 at 9:40 PM Priya Ch wrote: > Hello All, > > I am trying to extend SparkListener and post job ends trying to retrieve > job name to check the status of either success/failure and write to log > file. > > I couldn't find a way

Re: Should python-2 be supported in Spark 3.0?

2018-09-17 Thread Mark Hamstra
ully removing it: for > example, if Pandas and TensorFlow no longer support Python 2 past some > point, that might be a good point to remove it. > > Matei > > > On Sep 17, 2018, at 11:01 AM, Mark Hamstra > wrote: > > > > If we're going to do that, then we need to d

Re: Should python-2 be supported in Spark 3.0?

2018-09-17 Thread Mark Hamstra
spark versions supporting > Py2 past the point where Py2 is no longer receiving security patches > > > On Sun, Sep 16, 2018 at 12:26 PM Mark Hamstra > wrote: > >> We could also deprecate Py2 already in the 2.4.0 release. >> >> On Sat, Sep 15, 2018 at 11:46 A

Re: Should python-2 be supported in Spark 3.0?

2018-09-16 Thread Mark Hamstra
We could also deprecate Py2 already in the 2.4.0 release. On Sat, Sep 15, 2018 at 11:46 AM Erik Erlandson wrote: > In case this didn't make it onto this thread: > > There is a 3rd option, which is to deprecate Py2 for Spark-3.0, and remove > it entirely on a later 3.x release. > > On Sat, Sep

Re: [SPARK on MESOS] Avoid re-fetching Spark binary

2018-07-10 Thread Mark Hamstra
It's been done many times before by many organizations. Use Spark Job Server or Livy or create your own implementation of a similar long-running Spark Application. Creating a new Application for every Job is not the way to achieve low-latency performance. On Tue, Jul 10, 2018 at 4:18 AM wrote:

Re: [SPARK on MESOS] Avoid re-fetching Spark binary

2018-07-07 Thread Mark Hamstra
Essentially correct. The latency to start a Spark Job is nowhere close to 2-4 seconds under typical conditions. Creating a new Spark Application every time instead of running multiple Jobs in one Application is not going to lead to acceptable interactive or real-time performance, nor is that an

Re: [SPARK on MESOS] Avoid re-fetching Spark binary

2018-07-06 Thread Mark Hamstra
The latency to start a Spark Job is nowhere close to 2-4 seconds under typical conditions. You appear to be creating a new Spark Application everytime instead of running multiple Jobs in one Application. On Fri, Jul 6, 2018 at 3:12 AM Tien Dat wrote: > Dear Timothy, > > It works like a charm

Re: Spark horizontal scaling is not supported in which cluster mode? Ask

2018-05-21 Thread Mark Hamstra
Horizontal scaling is scaling across multiple, distributed computers (or at least OS instances). Local mode is, therefore, by definition not horizontally scalable since it just uses a configurable number of local threads. If the question actually asked "which cluster manager...?", then I have a

Re: [Spark 2.x Core] .collect() size limit

2018-04-28 Thread Mark Hamstra
spark.driver.maxResultSize http://spark.apache.org/docs/latest/configuration.html On Sat, Apr 28, 2018 at 8:41 AM, klrmowse wrote: > i am currently trying to find a workaround for the Spark application i am > working on so that it does not have to use .collect() > > but,

Re: Spark with Scala 2.12

2018-04-21 Thread Mark Hamstra
Even more to the point: http://apache-spark-developers-list.1001551.n3.nabble.com/Scala-2-12-support-td23833.html tldr; It's an item of discussion, but there is no imminent release of Spark that will use Scala 2.12. On Sat, Apr 21, 2018 at 2:44 AM, purijatin wrote: > I see

Re: Performance of Spark when the compute and storage are separated

2018-04-15 Thread Mark Hamstra
Keep forgetting to reply to user list... On Sun, Apr 15, 2018 at 1:58 PM, Mark Hamstra <m...@clearstorydata.com> wrote: > Sure, data locality all the way at the basic storage layer is the easy way > to avoid paying the costs of remote I/O. My point, though, is that that >

Re: Databricks Serverless

2017-11-13 Thread Mark Hamstra
This is not a Databricks forum. On Mon, Nov 13, 2017 at 3:18 PM, Benjamin Kim wrote: > I have a question about this. The documentation compares the concept > similar to BigQuery. Does this mean that we will no longer need to deal > with instances and just pay for execution

Re: Dependency error due to scala version mismatch in SBT and Spark 2.1

2017-10-16 Thread Mark Hamstra
rrect way to build Spark with sbt. Why ? > > > On Sun, Oct 15, 2017 at 11:54 PM, Mark Hamstra <m...@clearstorydata.com> > wrote: > >> I am building Spark using build.sbt. >> >> >> Which just gets me back to my original question: Why? This is not the >

Re: How can i remove the need for calling cache

2017-08-01 Thread Mark Hamstra
Very likely, much of the potential duplication is already being avoided even without calling cache/persist. When running the above code without `myrdd.cache`, have you looked at the Spark web UI for the Jobs? For at least one of them you will likely see that many Stages are marked as "skipped",

Re: Spark on Cloudera Configuration (Scheduler Mode = FAIR)

2017-07-20 Thread Mark Hamstra
xecutors. > > In the meantime, submitting another Spark Application (*Application* # B) > with the scheduler.mode as FAIR and dynamicallocation=true but it got only > one executor. " > > Thanks & Regards, > Gokula Krishnan* (Gokul)* > > On Thu, Jul 20, 2017 at 4:

Re: Spark on Cloudera Configuration (Scheduler Mode = FAIR)

2017-07-20 Thread Mark Hamstra
First, Executors are not allocated to Jobs, but rather to Applications. If you run multiple Jobs within a single Application, then each of the Tasks associated with Stages of those Jobs has the potential to run on any of the Application's Executors. Second, once a Task starts running on an

Re: a stage can belong to more than one job please?

2017-06-06 Thread Mark Hamstra
Yes, a Stage can be part of more than one Job. The jobIds field of Stage is used repeatedly in the DAGScheduler. On Tue, Jun 6, 2017 at 5:04 AM, 萝卜丝炒饭 <1427357...@qq.com> wrote: > Hi all, > > I read same code of spark about stage. > > The constructor of stage keep the first job ID the stage was

Re: Using SparkContext in Executors

2017-05-28 Thread Mark Hamstra
You can't do that. SparkContext and SparkSession can exist only on the Driver. On Sun, May 28, 2017 at 6:56 AM, Abdulfattah Safa wrote: > How can I use SparkContext (to create Spark Session or Cassandra Sessions) > in executors? > If I pass it as parameter to the foreach

Re: 2.2. release date ?

2017-05-23 Thread Mark Hamstra
I heard that once we reach release candidates it's not a question of time or a target date, but only whether blockers are resolved and the code is ready to release. On Tue, May 23, 2017 at 11:07 AM, kant kodali wrote: > Heard its end of this month (May) > > On Tue, May 23,

Re: scalastyle violation on mvn install but not on mvn package

2017-05-23 Thread Mark Hamstra
il but the > two replies are not even in the same "email conversation". > I don't know the mechanics of why posts do or don't show up via Nabble, but Nabble is neither the canonical archive nor the system of record for Apache mailing lists. > On Thu, May 4, 2017 at 8:11 PM,

Re: Multiple CSV libs causes issues spark 2.1

2017-05-09 Thread Mark Hamstra
deciding which > lib to use. > > On 9 May 2017 at 14:30, Mark Hamstra <m...@clearstorydata.com> wrote: > >> This looks more like a matter for Databricks support than spark-user. >> >> On Tue, May 9, 2017 at 2:02 PM, lucas.g...@gmail.com < >> lucas.g...@g

Re: Multiple CSV libs causes issues spark 2.1

2017-05-09 Thread Mark Hamstra
This looks more like a matter for Databricks support than spark-user. On Tue, May 9, 2017 at 2:02 PM, lucas.g...@gmail.com wrote: > df = spark.sqlContext.read.csv('out/df_in.csv') >> > > >> 17/05/09 15:51:29 WARN ObjectStore: Version information not found in >> metastore.

Re: scalastyle violation on mvn install but not on mvn package

2017-05-04 Thread Mark Hamstra
The check goal of the scalastyle plugin runs during the "verify" phase, which is between "package" and "install"; so running just "package" will not run scalastyle:check. On Thu, May 4, 2017 at 7:45 AM, yiskylee wrote: > ./build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0

Re: Securing Spark Job on Cluster

2017-04-28 Thread Mark Hamstra
spark.local.dir http://spark.apache.org/docs/latest/configuration.html On Fri, Apr 28, 2017 at 8:51 AM, Shashi Vishwakarma < shashi.vish...@gmail.com> wrote: > Yes I am using HDFS .Just trying to understand couple of point. > > There would be two kind of encryption which would be required. > >

Re: Spark fair scheduler pools vs. YARN queues

2017-04-05 Thread Mark Hamstra
scheduler pools relevant/useful in this context? > > On Wed, Apr 5, 2017 at 3:54 PM Mark Hamstra <m...@clearstorydata.com> > wrote: > >> grrr... s/your/you're/ >> >> On Wed, Apr 5, 2017 at 12:54 PM, Mark Hamstra <m...@clearstorydata.com> >> wrote: >&

Re: Spark fair scheduler pools vs. YARN queues

2017-04-05 Thread Mark Hamstra
grrr... s/your/you're/ On Wed, Apr 5, 2017 at 12:54 PM, Mark Hamstra <m...@clearstorydata.com> wrote: > Your mixing up different levels of scheduling. Spark's fair scheduler > pools are about scheduling Jobs, not Applications; whereas YARN queues with > Spark are about scheduli

Re: Spark fair scheduler pools vs. YARN queues

2017-04-05 Thread Mark Hamstra
Your mixing up different levels of scheduling. Spark's fair scheduler pools are about scheduling Jobs, not Applications; whereas YARN queues with Spark are about scheduling Applications, not Jobs. On Wed, Apr 5, 2017 at 12:27 PM, Nick Chammas wrote: > I'm having

Re: Spark shuffle files

2017-03-27 Thread Mark Hamstra
When the RDD using them goes out of scope. On Mon, Mar 27, 2017 at 3:13 PM, Ashwin Sai Shankar <ashan...@netflix.com> wrote: > Thanks Mark! follow up question, do you know when shuffle files are > usually un-referenced? > > On Mon, Mar 27, 2017 at 2:35 PM, Mark Hamstra <m.

Re: Spark shuffle files

2017-03-27 Thread Mark Hamstra
Shuffle files are cleaned when they are no longer referenced. See https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/ContextCleaner.scala On Mon, Mar 27, 2017 at 12:38 PM, Ashwin Sai Shankar < ashan...@netflix.com.invalid> wrote: > Hi! > > In spark on yarn, when are

Re: Can't transform RDD for the second time

2017-02-28 Thread Mark Hamstra
foreachPartition is not a transformation; it is an action. If you want to transform an RDD using an iterator in each partition, then use mapPartitions. On Tue, Feb 28, 2017 at 8:17 PM, jeremycod wrote: > Hi, > > I'm trying to transform one RDD two times. I'm using

Fwd: Will Spark ever run the same task at the same time

2017-02-20 Thread Mark Hamstra
First, the word you are looking for is "straggler", not "strangler" -- very different words. Second, "idempotent" doesn't mean "only happens once", but rather "if it does happen more than once, the effect is no different than if it only happened once". It is possible to insert a nearly limitless

Re: is dataframe thread safe?

2017-02-13 Thread Mark Hamstra
If you update the data, then you don't have the same DataFrame anymore. If you don't do like Assaf did, caching and forcing evaluation of the DataFrame before using that DataFrame concurrently, then you'll still get consistent and correct results, but not necessarily efficient results. If the

Re: can I use Spark Standalone with HDFS but no YARN

2017-02-03 Thread Mark Hamstra
yes On Fri, Feb 3, 2017 at 10:08 PM, kant kodali wrote: > can I use Spark Standalone with HDFS but no YARN? > > Thanks! >

Re: Having multiple spark context

2017-01-29 Thread Mark Hamstra
More than one Spark Context in a single Application is not supported. On Sun, Jan 29, 2017 at 9:08 PM, wrote: > Hi, > > > > I have a requirement in which, my application creates one Spark context in > Distributed mode whereas another Spark context in local mode. > >

Re: DAG Visualization option is missing on Spark Web UI

2017-01-28 Thread Mark Hamstra
Try selecting a particular Job instead of looking at the summary page for all Jobs. On Sat, Jan 28, 2017 at 4:25 PM, Md. Rezaul Karim < rezaul.ka...@insight-centre.org> wrote: > Hi Jacek, > > I tried accessing Spark web UI on both Firefox and Google Chrome browsers > with ad blocker enabled. I

Re:

2017-01-21 Thread Mark Hamstra
I wouldn't say that Executors are dumb, but there are some pretty clear divisions of concepts and responsibilities across the different pieces of the Spark architecture. A Job is a concept that is completely unknown to an Executor, which deals instead with just the Tasks that it is given. So you

Re: Spark and Kafka integration

2017-01-12 Thread Mark Hamstra
See "API compatibility" in http://spark.apache.org/versioning-policy.html While code that is annotated as Experimental is still a good faith effort to provide a stable and useful API, the fact is that we're not yet confident enough that we've got the public API in exactly the form that we want to

Re: Is there any scheduled release date for Spark 2.1.0?

2016-12-28 Thread Mark Hamstra
; Wed > Dec 28 20:01:10 UTC 2016 > 2.2.0-SNAPSHOT/ > <https://repository.apache.org/content/groups/snapshots/org/apache/spark/spark-sql_2.11/2.2.0-SNAPSHOT/> > Wed > Dec 28 19:12:38 UTC 2016 > > What's with 2.1.1-SNAPSHOT? Is that version about to be released as well? > > Than

Re: Is there any scheduled release date for Spark 2.1.0?

2016-12-28 Thread Mark Hamstra
The v2.1.0 tag is there: https://github.com/apache/spark/tree/v2.1.0 On Wed, Dec 28, 2016 at 2:04 PM, Koert Kuipers wrote: > seems like the artifacts are on maven central but the website is not yet > updated. > > strangely the tag v2.1.0 is not yet available on github. i

Re: Infinite Loop in Spark

2016-10-27 Thread Mark Hamstra
Using a single SparkContext for an extended period of time is how long-running Spark Applications such as the Spark Job Server work ( https://github.com/spark-jobserver/spark-jobserver). It's an established pattern. On Thu, Oct 27, 2016 at 11:46 AM, Gervásio Santos wrote: >

Re: previous stage results are not saved?

2016-10-17 Thread Mark Hamstra
There is no need to do that if 1) the stage that you are concerned with either made use of or produced MapOutputs/shuffle files; 2) reuse of those shuffle files (which may very well be in the OS buffer cache of the worker nodes) is sufficient for your needs; 3) the relevant Stage objects haven't

Re: Is executor computing time affected by network latency?

2016-09-23 Thread Mark Hamstra
> > The best network results are achieved when Spark nodes share the same > hosts as Hadoop or they happen to be on the same subnet. > That's only true for those portions of a Spark execution pipeline that are actually reading from HDFS. If you're re-using an RDD for which the needed shuffle

Re: Spark 2.0.0 won't let you create a new SparkContext?

2016-09-13 Thread Mark Hamstra
It sounds like you should be writing an application and not trying to force the spark-shell to do more than what it was intended for. On Tue, Sep 13, 2016 at 11:53 AM, Kevin Burton wrote: > I sort of agree but the problem is that some of this should be code. > > Some of our

Re: Spark scheduling mode

2016-09-02 Thread Mark Hamstra
And, no, Spark's scheduler will not preempt already running Tasks. In fact, just killing running Tasks for any reason is trickier than we'd like it to be, so it isn't done by default: https://issues.apache.org/jira/browse/SPARK-17064 On Fri, Sep 2, 2016 at 11:34 AM, Mark Hamstra &l

Re: Spark scheduling mode

2016-09-02 Thread Mark Hamstra
I right? > > Thank you > ------ > *From:* Mark Hamstra <m...@clearstorydata.com> > *Sent:* Thursday, September 1, 2016 8:44:10 PM > > *To:* enrico d'urso > *Cc:* user@spark.apache.org > *Subject:* Re: Spark scheduling mode > >

Re: Spark scheduling mode

2016-09-01 Thread Mark Hamstra
eduled in round robin way, > am I right? > > -- > *From:* Mark Hamstra <m...@clearstorydata.com> > *Sent:* Thursday, September 1, 2016 8:19:44 PM > *To:* enrico d'urso > *Cc:* user@spark.apache.org > *Subject:* Re: Spark scheduling mode > > The default

Re: Spark scheduling mode

2016-09-01 Thread Mark Hamstra
ault pool? > I mean, round robin for the jobs that belong to the default pool. > > Cheers, > -- > *From:* Mark Hamstra <m...@clearstorydata.com> > *Sent:* Thursday, September 1, 2016 7:24:54 PM > *To:* enrico d'urso > *Cc:* user@spark.apache.org

Re: Spark scheduling mode

2016-09-01 Thread Mark Hamstra
Just because you've flipped spark.scheduler.mode to FAIR, that doesn't mean that Spark can magically configure and start multiple scheduling pools for you, nor can it know to which pools you want jobs assigned. Without doing any setup of additional scheduling pools or assigning of jobs to pools,

Re: What do I loose if I run spark without using HDFS or Zookeeper?

2016-08-25 Thread Mark Hamstra
>> >>> Zookeeper is not just about high availability. It is used in Spark >>> Streaming with Kafka, it is also used with Hive for concurrency. It is also >>> a distributed locking system. >>> >>> HTH >>> >>> Dr Mich Talebzadeh

Re: What do I loose if I run spark without using HDFS or Zookeeper?

2016-08-25 Thread Mark Hamstra
s/playing a role/paying a role/ On Thu, Aug 25, 2016 at 12:51 PM, Mark Hamstra <m...@clearstorydata.com> wrote: > One way you can start to make this make more sense, Sean, is if you > exploit the code/data duality so that the non-distributed data that you are > sending out

Re: What do I loose if I run spark without using HDFS or Zookeeper?

2016-08-25 Thread Mark Hamstra
One way you can start to make this make more sense, Sean, is if you exploit the code/data duality so that the non-distributed data that you are sending out from the driver is actually paying a role more like code (or at least parameters.) What is sent from the driver to an Executer is then used

Re: Spark 2.0.1 / 2.1.0 on Maven

2016-08-09 Thread Mark Hamstra
mer:* Use it at your own risk. Any and all responsibility for >> any loss, damage or destruction of data or any other property which may >> arise from relying on this email's technical content is explicitly >> disclaimed. The author will in no case be liable for any monetary damages >

Re: Spark 2.0.1 / 2.1.0 on Maven

2016-08-09 Thread Mark Hamstra
What are you expecting to find? There currently are no releases beyond Spark 2.0.0. On Tue, Aug 9, 2016 at 9:55 AM, Jestin Ma wrote: > If we want to use versions of Spark beyond the official 2.0.0 release, > specifically on Maven + Java, what steps should we take to

Re: SPARK Exception thrown in awaitResult

2016-07-28 Thread Mark Hamstra
Don't use Spark 2.0.0-preview. That was a preview release with known issues, and was intended to be used only for early, pre-release testing purpose. Spark 2.0.0 is now released, and you should be using that. On Thu, Jul 28, 2016 at 3:48 AM, Carlo.Allocca wrote: >

Re: standalone mode only supports FIFO scheduler across applications ? still in spark 2.0 time ?

2016-07-15 Thread Mark Hamstra
Nothing has changed in that regard, nor is there likely to be "progress", since more sophisticated or capable resource scheduling at the Application level is really beyond the design goals for standalone mode. If you want more in the way of multi-Application resource scheduling, then you should

Re: What is the interpretation of Cores in Spark doc

2016-06-16 Thread Mark Hamstra
h Talebzadeh > > > > LinkedIn * > https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw > <https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw>* > > > > http://talebzadehmich.wordpress.com > > > >

Re: What is the interpretation of Cores in Spark doc

2016-06-16 Thread Mark Hamstra
> 3 full cores with 2 threads each. Rational is due to the way OS >> dispatching works on ‘logical’ processors vs. cores and POSIX threaded >> applications. >> >> HTH >> >> Dr Mich Talebzadeh >> >> >> LinkedIn * >> https://www.linkedin.com/pr

Re: What is the interpretation of Cores in Spark doc

2016-06-16 Thread Mark Hamstra
is due to the way OS >> dispatching works on ‘logical’ processors vs. cores and POSIX threaded >> applications. >> >> HTH >> >> Dr Mich Talebzadeh >> >> >> LinkedIn * >> https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCC

Re: What is the interpretation of Cores in Spark doc

2016-06-13 Thread Mark Hamstra
I don't know what documentation you were referring to, but this is clearly an erroneous statement: "Threads are virtual cores." At best it is terminology abuse by a hardware manufacturer. Regardless, Spark can't get too concerned about how any particular hardware vendor wants to refer to the

Re: Spark UI doesn't give visibility on which stage job actually failed (due to lazy eval nature)

2016-05-25 Thread Mark Hamstra
s a black box then it's fine but when you > have large number of people on this site complaining about OOM and shuffle > error all the time you need to start providing some transparency to > address that. > > Thanks > > > On Wed, May 25, 2016 at 6:41 PM, Mark Hamstra <m...@c

Re: Spark UI doesn't give visibility on which stage job actually failed (due to lazy eval nature)

2016-05-25 Thread Mark Hamstra
You appear to be misunderstanding the nature of a Stage. Individual transformation steps such as `map` do not define the boundaries of Stages. Rather, a sequence of transformations in which there is only a NarrowDependency between each of the transformations will be pipelined into a single Stage.

Re: Apache Flink

2016-04-17 Thread Mark Hamstra
To be fair, the Stratosphere project from which Flink springs was started as a collaborative university research project in Germany about the same time that Spark was first released as Open Source, so they are near contemporaries rather than Flink having been started only well after Spark was an

Re: YARN vs Standalone Spark Usage in production

2016-04-14 Thread Mark Hamstra
That's also available in standalone. On Thu, Apr 14, 2016 at 12:47 PM, Alexander Pivovarov wrote: > Spark on Yarn supports dynamic resource allocation > > So, you can run several spark-shells / spark-submits / spark-jobserver / > zeppelin on one cluster without defining

Re: Spark GUI, Workers and Executors

2016-04-09 Thread Mark Hamstra
https://spark.apache.org/docs/latest/cluster-overview.html On Sat, Apr 9, 2016 at 12:28 AM, Ashok Kumar wrote: > On Spark GUI I can see the list of Workers. > > I always understood that workers are used by executors. > > What is the relationship between workers and

Re: Executor shutdown hooks?

2016-04-06 Thread Mark Hamstra
Why would the Executors shutdown when the Job is terminated? Executors are bound to Applications, not Jobs. Furthermore, unless spark.job.interruptOnCancel is set to true, canceling the Job at the Application and DAGScheduler level won't actually interrupt the Tasks running on the Executors. If

Re: Spark and N-tier architecture

2016-03-29 Thread Mark Hamstra
LinkedIn * > https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw > <https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw>* > > > > http://talebzadehmich.wordpress.com > > > > On 30 March 2016 at 00:22, Mark Hamstra <m...@clearstoryda

Re: Spark and N-tier architecture

2016-03-29 Thread Mark Hamstra
Yes and no. The idea of n-tier architecture is about 20 years older than Spark and doesn't really apply to Spark as n-tier was original conceived. If the n-tier model helps you make sense of some things related to Spark, then use it; but don't get hung up on trying to force a Spark architecture

Re: No active SparkContext

2016-03-24 Thread Mark Hamstra
You seem to be confusing the concepts of Job and Application. A Spark Application has a SparkContext. A Spark Application is capable of running multiple Jobs, each with its own ID, visible in the webUI. On Thu, Mar 24, 2016 at 6:11 AM, Max Schmidt wrote: > Am 24.03.2016 um

Re: What's the benifit of RDD checkpoint against RDD save

2016-03-23 Thread Mark Hamstra
On Wed, Mar 23, 2016 at 7:38 PM, Ted Yu <yuzhih...@gmail.com> wrote: > bq. when I get the last RDD > If I read Todd's first email correctly, the computation has been done. > I could be wrong. > > On Wed, Mar 23, 2016 at 7:34 PM, Mark Hamstra <m...@clearstorydata.com> > wr

Re: What's the benifit of RDD checkpoint against RDD save

2016-03-23 Thread Mark Hamstra
Neither of you is making any sense to me. If you just have an RDD for which you have specified a series of transformations but you haven't run any actions, then neither checkpointing nor saving makes sense -- you haven't computed anything yet, you've only written out the recipe for how the

Re: Error using collectAsMap() in scala

2016-03-20 Thread Mark Hamstra
You're not getting what Ted is telling you. Your `dict` is an RDD[String] -- i.e. it is a collection of a single value type, String. But `collectAsMap` is only defined for PairRDDs that have key-value pairs for their data elements. Both a key and a value are needed to collect into a Map[K, V].

Re: Spark UI Completed Jobs

2016-03-15 Thread Mark Hamstra
It's not just if the RDD is explicitly cached, but also if the map outputs for stages have been materialized into shuffle files and are still accessible through the map output tracker. Because of that, explicitly caching RDD actions often gains you little or nothing, since even without a call to

Re: Can we use spark inside a web service?

2016-03-10 Thread Mark Hamstra
icular architecture work in production. i > would love for someone to prove otherwise. > > On Thu, Mar 10, 2016 at 5:44 PM, Mark Hamstra <m...@clearstorydata.com> > wrote: > >> For example, if you're looking to scale out to 1000 concurrent requests, >>> this is 1000 co

Re: Can we use spark inside a web service?

2016-03-10 Thread Mark Hamstra
> > For example, if you're looking to scale out to 1000 concurrent requests, > this is 1000 concurrent Spark jobs. This would require a cluster with 1000 > cores. This doesn't make sense. A Spark Job is a driver/DAGScheduler concept without any 1:1 correspondence between Worker cores and Jobs.

Re: Spark on RAID

2016-03-08 Thread Mark Hamstra
One issue is that RAID levels providing data replication are not necessary since HDFS already replicates blocks on multiple nodes. On Tue, Mar 8, 2016 at 8:45 AM, Alex Kozlov wrote: > Parallel disk IO? But the effect should be less noticeable compared to > Hadoop which

Re: Understanding the Web_UI 4040

2016-03-07 Thread Mark Hamstra
There's probably nothing wrong other than a glitch in the reporting of Executor state transitions to the UI -- one of those low-priority items I've been meaning to look at for awhile On Mon, Mar 7, 2016 at 12:15 AM, Sonal Goyal wrote: > Maybe check the worker logs to

Re: Fair scheduler pool details

2016-03-02 Thread Mark Hamstra
alone deployment (it is slightly mentioned in SPARK-9882, but it seems > to be abandoned). Do you know if there is such an activity? > > -- > Be well! > Jean Morozov > > On Sun, Feb 21, 2016 at 4:32 AM, Mark Hamstra <m...@clearstorydata.com> > wrote: > >> It's 2

Re: Fair scheduler pool details

2016-02-20 Thread Mark Hamstra
It's 2 -- and it's pretty hard to point to a line of code, a method, or even a class since the scheduling of Tasks involves a pretty complex interaction of several Spark components -- mostly the DAGScheduler, TaskScheduler/TaskSchedulerImpl, TaskSetManager, Schedulable and Pool, as well as the

Re: Creating HiveContext in Spark-Shell fails

2016-02-15 Thread Mark Hamstra
Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.0.0-SNAPSHOT /_/ Using Scala version 2.11.7 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_72) Type in expressions to have them evaluated. Type

Re: Spark 2.0.0 release plan

2016-01-29 Thread Mark Hamstra
https://github.com/apache/spark/pull/10608 On Fri, Jan 29, 2016 at 11:50 AM, Jakob Odersky wrote: > I'm not an authoritative source but I think it is indeed the plan to > move the default build to 2.11. > > See this discussion for more detail > >

Re: Spark DataFrame Catalyst - Another Oracle like query optimizer?

2016-01-25 Thread Mark Hamstra
What do you think is preventing you from optimizing your own RDD-level transformations and actions? AFAIK, nothing that has been added in Catalyst precludes you from doing that. The fact of the matter is, though, that there is less type and semantic information available to Spark from the raw

Re: Reuse Executor JVM across different JobContext

2016-01-17 Thread Mark Hamstra
-dev What do you mean by JobContext? That is a Hadoop mapreduce concept, not Spark. On Sun, Jan 17, 2016 at 7:29 AM, Jia Zou wrote: > Dear all, > > Is there a way to reuse executor JVM across different JobContexts? Thanks. > > Best Regards, > Jia >

Re: Reuse Executor JVM across different JobContext

2016-01-17 Thread Mark Hamstra
t; Thanks, Mark. Then, I guess JobServer can fundamentally solve my problem, > so that jobs can be submitted at different time and still share RDDs. > > Best Regards, > Jia > > > On Jan 17, 2016, at 3:44 PM, Mark Hamstra <m...@clearstorydata.com> wrote: > > Th

Re: Reuse Executor JVM across different JobContext

2016-01-17 Thread Mark Hamstra
tion will be executed by the executor. After the application > runs to completion. The executor process will be killed. > But I hope that all applications submitted can run in the same executor, > can JobServer do that? If so, it’s really good news! > > Best Regards, > Jia > >

Re: simultaneous actions

2016-01-17 Thread Mark Hamstra
Same SparkContext means same pool of Workers. It's up to the Scheduler, not the SparkContext, whether the exact same Workers or Executors will be used to calculate simultaneous actions against the same RDD. It is likely that many of the same Workers and Executors will be used as the Scheduler

Re: Reuse Executor JVM across different JobContext

2016-01-17 Thread Mark Hamstra
<jacqueline...@gmail.com> wrote: > Hi, Mark, sorry, I mean SparkContext. > I mean to change Spark into running all submitted jobs (SparkContexts) in > one executor JVM. > > Best Regards, > Jia > > On Sun, Jan 17, 2016 at 2:21 PM, Mark Hamstra <m...@clearstorydata.com&

Re: simultaneous actions

2016-01-17 Thread Mark Hamstra
fle files is always a nice surprise to me > > On Sun, Jan 17, 2016 at 3:17 PM, Mark Hamstra <m...@clearstorydata.com> > wrote: > >> Same SparkContext means same pool of Workers. It's up to the Scheduler, >> not the SparkContext, whether the exact same Workers or

Re: spark 1.6 Issue

2016-01-06 Thread Mark Hamstra
It's not a bug, but a larger heap is required with the new UnifiedMemoryManager: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala#L172 On Wed, Jan 6, 2016 at 6:35 AM, kali.tumm...@gmail.com < kali.tumm...@gmail.com> wrote: > Hi

Re: Can spark.scheduler.pool be applied globally ?

2016-01-05 Thread Mark Hamstra
I don't understand. If you're using fair scheduling and don't set a pool, the default pool will be used. On Tue, Jan 5, 2016 at 1:57 AM, Jeff Zhang wrote: > > It seems currently spark.scheduler.pool must be set as localProperties > (associate with thread). Any reason why

Re: Can spark.scheduler.pool be applied globally ?

2016-01-05 Thread Mark Hamstra
> Right, I can override the root pool in configuration file, Thanks Mark. > > On Wed, Jan 6, 2016 at 8:45 AM, Mark Hamstra <m...@clearstorydata.com> > wrote: > >> Just configure with >> FAIR in fairscheduler.xml (or >> in spark.scheduler.allocation.file if

Re: Can spark.scheduler.pool be applied globally ?

2016-01-05 Thread Mark Hamstra
it clearly. What I want is the default pool is fair > scheduling. But seems if I want to use fair scheduling now, I have to set > spark.scheduler.pool explicitly. > > On Wed, Jan 6, 2016 at 2:03 AM, Mark Hamstra <m...@clearstorydata.com> > wrote: > >> I don't understan

Re: SparkContext.cancelJob - what part of Spark uses it? Nothing in webUI to kill jobs?

2015-12-17 Thread Mark Hamstra
://twitter.com/jaceklaskowski > Upvote at http://stackoverflow.com/users/1305344/jacek-laskowski > > > On Wed, Dec 16, 2015 at 10:55 AM, Mark Hamstra <m...@clearstorydata.com> > wrote: > > It can be used, and is used in user code, but it isn't always as > > straig

Re: SparkContext.cancelJob - what part of Spark uses it? Nothing in webUI to kill jobs?

2015-12-16 Thread Mark Hamstra
It can be used, and is used in user code, but it isn't always as straightforward as you might think. This is mostly because a Job often isn't a Job -- or rather it is more than one Job. There are several RDD transformations that aren't lazy, so they end up launching "hidden" Jobs that you may

Re: RE: Spark assembly in Maven repo?

2015-12-10 Thread Mark Hamstra
No, publishing a spark assembly jar is not fine. See the doc attached to https://issues.apache.org/jira/browse/SPARK-11157 and be aware that a likely goal of Spark 2.0 will be the elimination of assemblies. On Thu, Dec 10, 2015 at 11:19 PM, fightf...@163.com wrote: > Using

Re: is Multiple Spark Contexts is supported in spark 1.5.0 ?

2015-12-04 Thread Mark Hamstra
Where it could start to make some sense is if you wanted a single application to be able to work with more than one Spark cluster -- but that's a pretty weird or unusual thing to do, and I'm pretty sure it wouldn't work correctly at present. On Fri, Dec 4, 2015 at 11:10 AM, Michael Armbrust

Re: Low Latency SQL query

2015-12-01 Thread Mark Hamstra
ed in RDDs and need low latency it may > not be a good fit. > > M > > On Dec 1, 2015, at 7:23 PM, Andrés Ivaldi <iaiva...@gmail.com> wrote: > > Ok, so latency problem is being generated because I'm using SQL as source? > how about csv, hive, or another source? > > On

Re: Low Latency SQL query

2015-12-01 Thread Mark Hamstra
other solution. On Tue, Dec 1, 2015 at 4:23 PM, Andrés Ivaldi <iaiva...@gmail.com> wrote: > Ok, so latency problem is being generated because I'm using SQL as source? > how about csv, hive, or another source? > > On Tue, Dec 1, 2015 at 9:18 PM, Mark Hamstra <m...@clears

  1   2   >