Re: [DISCUSS] Deprecate DStream in 3.4

2023-01-12 Thread Tathagata Das
+1 On Thu, Jan 12, 2023 at 7:46 PM Hyukjin Kwon wrote: > +1 > > On Fri, 13 Jan 2023 at 08:51, Jungtaek Lim > wrote: > >> bump for more visibility. >> >> On Wed, Jan 11, 2023 at 12:20 PM Jungtaek Lim < >> kabhwan.opensou...@gmail.com> wrote: >> >>> Hi dev, >>> >>> I'd like to propose the depreca

Re: dropDuplicates and watermark in structured streaming

2020-02-27 Thread Tathagata Das
1. Yes. All times in event time, not processing time. So you may get 10AM event time data at 11AM processing time, but it will still be compared again all data within 9-10AM event times. 2. Show us your code. On Thu, Feb 27, 2020 at 2:30 AM lec ssmi wrote: > Hi: > I'm new to structured stre

Re: dropDuplicates and watermark in structured streaming

2020-02-28 Thread Tathagata Das
lec ssmi wrote: > Such as : > df.withWarmark("time","window > size").dropDulplicates("id").withWatermark("time","real > watermark").groupBy(window("time","window size","window > size")).agg(count(&

Re: [vote] Apache Spark 3.0 RC3

2020-06-09 Thread Tathagata Das
+1 (binding) On Tue, Jun 9, 2020 at 5:27 PM Burak Yavuz wrote: > +1 > > Best, > Burak > > On Tue, Jun 9, 2020 at 1:48 PM Shixiong(Ryan) Zhu > wrote: > >> +1 (binding) >> >> Best Regards, >> Ryan >> >> >> On Tue, Jun 9, 2020 at 4:24 AM Wenchen Fan wrote: >> >>> +1 (binding) >>> >>> On Tue, Jun

Re: In structured streamin, multiple streaming aggregations are not yet supported.

2017-11-28 Thread Tathagata Das
Hello, What do you mean by multiple streaming aggregations? Something like this is already supported. *df.groupBy("key").agg(min("colA"), max("colB"), avg("colC"))* But the following is not supported. *df.groupBy("key").agg(min("colA").as("min")).groupBy("min").count()* In other words, multipl

Metrics for monitoring Structured Streaming apps

2016-10-03 Thread Tathagata Das
Hey all, I have been working on adding operational metrics for monitoring the health and performance of Structured Streaming applications. The detailed design and the WIP Github PR is here. *JIRA *- SPARK-17731 *Design Doc* - https://docs.google

Watermarking in Structured Streaming to drop late data

2016-10-26 Thread Tathagata Das
Hey all, We are planning implement watermarking in Structured Streaming that would allow us handle late, out-of-order data better. Specially, when we are aggregating over windows on event-time, we currently can end up keeping unbounded amount data as state. We want to define watermarks on the even

Re: Watermarking in Structured Streaming to drop late data

2016-10-27 Thread Tathagata Das
Spark Developers List] [mailto: >> ml-node+[hidden email] >> <http:///user/SendEmail.jtp?type=node&node=19591&i=0>] >> *Sent:* Thursday, October 27, 2016 3:04 AM >> *To:* Mendelson, Assaf >> *Subject:* Re: Watermarking in Structured Streaming to d

Re: Watermarking in Structured Streaming to drop late data

2016-10-27 Thread Tathagata Das
node=19592&i=1>[hidden email] > <http://user/SendEmail.jtp?type=node&node=19591&i=0>] > *Sent:* Thursday, October 27, 2016 3:04 AM > *To:* Mendelson, Assaf > *Subject:* Re: Watermarking in Structured Streaming to drop late data > > > > And the JIRA: https:/

Re: REST api for monitoring Spark Streaming

2016-11-07 Thread Tathagata Das
This may be a good addition. I suggest you read our guidelines on contributing code to Spark. https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-PreparingtoContributeCodeChanges Its long document but it should have everything for you to figure out how to c

Re: [VOTE] Release Apache Spark 2.0.2 (RC3)

2016-11-10 Thread Tathagata Das
+1 binding On Thu, Nov 10, 2016 at 6:05 PM, Kousuke Saruta wrote: > +1 (non-binding) > > > On 2016年11月08日 15:09, Reynold Xin wrote: > >> Please vote on releasing the following candidate as Apache Spark version >> 2.0.2. The vote is open until Thu, Nov 10, 2016 at 22:00 PDT and passes if >> a maj

Re: Structured Streaming Schema Issue

2017-02-01 Thread Tathagata Das
You should make sure that schema of the streaming Dataset returned by `readStream`, and the schema of the DataFrame returned by the sources getBatch. On Wed, Feb 1, 2017 at 3:25 PM, Sam Elamin wrote: > Hi All > > I am writing a bigquery connector here >

Re: Structured Streaming Schema Issue

2017-02-01 Thread Tathagata Das
by the time I get to the addBatch method, > the dataframe there has an incorrect Schema > > So Im skeptical about the issue being prior to the readStream since the > output dataframe has the correct Schema > > > Am I missing something completely obvious? > > Regards >

Re: Structured Streaming Schema Issue

2017-02-01 Thread Tathagata Das
gt; have been turned to strings. > > > > On Wed, Feb 1, 2017 at 11:40 PM, Tathagata Das < > tathagata.das1...@gmail.com> wrote: > >> I am assuming that you have written your own BigQuerySource (i dont see >> that code in the link you posted). In that source, you must

Re: Any plans for making StateStore pluggable?

2017-05-09 Thread Tathagata Das
Thank you for creating the JIRA. I am working towards making it configurable very soon. On Tue, May 9, 2017 at 4:12 PM, Yogesh Mahajan wrote: > Hi Team, > > Any plans to make the StateStoreProvider/StateStore in structured > streaming pluggable ? > Currently StateStore#loadedProviders has only o

Re: Questions about Stateful Operations in SS

2017-07-26 Thread Tathagata Das
Hello Lubo, The idea of timeouts is to make a best-effort and last-resort effort to process a key, when it has not received data for a while. With processing time timeout is 1 minute, the system guarantees that it will not timeout unless at least 1 minute has passed. Defining a precise timing on w

Re: Time window on Processing Time

2017-08-29 Thread Tathagata Das
Yes, it can be! There is a sql function called current_timestamp() which is self-explanatory. So I believe you should be able to do something like import org.apache.spark.sql.functions._ ds.withColumn("processingTime", current_timestamp()) .groupBy(window("processingTime", "1 minute")) .count

Re: [VOTE] Designating maintainers for some Spark components

2014-11-07 Thread Tathagata Das
+1 (binding) I agree with the proposal that it just formalizes what we have been doing till now, and will increase the efficiency and focus of the review process. To address Davies' concern, I agree coding style is often a hot topic of contention. But that is just an indication that our processes

Re: Replacing Spark's native scheduler with Sparrow

2014-11-08 Thread Tathagata Das
Let me chime in on the discussion as well. Spark Streaming is another usecase where the scheduler's task-launching throughput and task-latency can limit the batch interval and the overall latencies achievable by Spark Streaming. Lets say we want to do batches of 20 ms (for achieve end-to-end latenc

Re: Replacing Spark's native scheduler with Sparrow

2014-11-10 Thread Tathagata Das
Too bad Nick, I dont have anything immediately ready that tests Spark Streaming with those extreme settings. :) On Mon, Nov 10, 2014 at 9:56 AM, Nicholas Chammas wrote: > On Sun, Nov 9, 2014 at 1:51 AM, Tathagata Das > wrote: >> >> This causes a scalability vs. latency tradeo

Re: HA support for Spark

2014-12-11 Thread Tathagata Das
Spark Streaming essentially does this by saving the DAG of DStreams, which can deterministically regenerate the DAG of RDDs upon recovery from failure. Along with that the progress information (which batches have finished, which batches are queued, etc.) is also saved, so that upon recovery the sys

Re: Which committers care about Kafka?

2014-12-29 Thread Tathagata Das
Hey all, Some wrap up thoughts on this thread. Let me first reiterate what Patrick said, that Kafka is super super important as it forms the largest fraction of Spark Streaming user base. So we really want to improve the Kafka + Spark Streaming integration. To this end, some of the things that ne

Re: Spark Streaming Data flow graph

2015-01-05 Thread Tathagata Das
Hey François, Well, at a high-level here is what I thought about the diagram. - ReceiverSupervisor handles only one Receiver. - BlockGenerator is part of ReceiverSupervisor not ReceivedBlockHandler - The blocks are inserted in BlockManager and if activated, WriteAheadLogManager in parallel, not t

Re: missing document of several messages in actor-based receiver?

2015-01-09 Thread Tathagata Das
It was not really mean to be hidden. So its essentially the case of the documentation being insufficient. This code has not gotten much attention for a while, so it could have a bugs. If you find any and submit a fix for them, I am happy to take a look! TD On Thu, Jan 8, 2015 at 6:33 PM, Nan Zhu

Re: When will Spark Streaming supports Kafka-simple consumer API?

2015-02-04 Thread Tathagata Das
1. There is already a third-party low-level kafka receiver - http://spark-packages.org/package/5 2. There is a new experimental Kafka stream that will be available in Spark 1.3 release. This is based on the low level API, and might suffice your purpose. JIRA - https://issues.apache.org/jira/browse/

Re: [VOTE] Release Apache Spark 1.3.0 (RC1)

2015-02-23 Thread Tathagata Das
Hey all, I found a major issue where JobProgressListener (a listener used to keep track of jobs for the web UI) never forgets stages in one of its data structures. This is a blocker for long running applications. https://issues.apache.org/jira/browse/SPARK-5967 I am testing a fix for this right n

Re: [VOTE] Release Apache Spark 1.3.0 (RC3)

2015-03-06 Thread Tathagata Das
To add to what Patrick said, the only reason that those JIRAs are marked as Blockers (at least I can say for myself) is so that they are at the top of the JIRA list signifying that these are more *immediate* issues than all the Critical issues. To make it less confusing for the community voting, we

Re: Spark Streaming - received block allocation to batch

2015-03-11 Thread Tathagata Das
See responses inline. On Wed, Mar 11, 2015 at 6:58 AM, Zoltán Zvara wrote: > I'm trying to understand the block allocation mechanism Spark uses to > generate batch jobs and a JobSet. > > The JobGenerator.generateJobs tries to allocate received blocks to batch, > effectively in ReceivedBlockTrack

Re: Spark 2.0: Rearchitecting Spark for Mobile, Local, Social

2015-04-01 Thread Tathagata Das
This is a significant effort that Reynold has undertaken, and I am super glad to see that it's finally taking a concrete form. Would love to see what the community thinks about the idea. TD On Wed, Apr 1, 2015 at 3:11 AM, Reynold Xin wrote: > Hi Spark devs, > > I've spent the last few months in

Re: Support for cycles in spark streaming topology ?????

2015-04-02 Thread Tathagata Das
Just to add to that, DStream.transform allows you do to arbitrary RDD-to-RDD function. Inside that you can do iterative RDD operations as well. On Thu, Apr 2, 2015 at 6:27 AM, Sean Owen wrote: > You can have diamonds but not cycles in the dependency graph. > > But what you are describing really

Re: Spark + Kinesis

2015-04-06 Thread Tathagata Das
| >>> is the name of the Kinesis stream | is the >>> endpoint of the Kinesis service | (e.g. >>> https://kinesis.us-east-1.amazonaws.com >>> <https://kinesis.us-east-1.amazonaws.com>)"&quo

Re: Which method do you think is better for making MIN_REMEMBER_DURATION configurable?

2015-04-08 Thread Tathagata Das
Approach 2 is definitely better :) Can you tell us more about the use case why you want to do this? TD On Wed, Apr 8, 2015 at 1:44 AM, Emre Sevinc wrote: > Hello, > > This is about SPARK-3276 and I want to make MIN_REMEMBER_DURATION (that is > now a constant) a variable (configurable, with a d

Re: Spark Streaming updatyeStateByKey throws OutOfMemory Error

2015-04-22 Thread Tathagata Das
It could very well be that your executor memory is not enough to store the state RDDs AND operate on the data. 1G per executor is quite low. Definitely give more memory. And have you tried increasing the number of partitions (specify number of partitions in updateStateByKey) ? On Wed, Apr 22, 2015

Re: Speeding up Spark build during development

2015-05-04 Thread Tathagata Das
In addition to Michael suggestion, in my SBT workflow I also use "~" to automatically kickoff build and unit test. For example, sbt/sbt "~streaming/test-only *BasicOperationsSuite*" It will automatically detect any file changes in the project and start of the compilation and testing. So my full w

Re: Spark Streaming - Design considerations/Knobs

2015-05-20 Thread Tathagata Das
Correcting the ones that are incorrect or incomplete. BUT this is good list for things to remember about Spark Streaming. On Wed, May 20, 2015 at 3:40 AM, Hemant Bhanawat wrote: > Hi, > > I have compiled a list (from online sources) of knobs/design > considerations that need to be taken care of

Re: Spark Streaming with Tachyon : Data Loss on Receiver Failure due to WAL error

2015-05-21 Thread Tathagata Das
Looks like somehow the file size reported by the FSInputDStream of Tachyon's FileSystem interface, is returning zero. On Mon, May 11, 2015 at 4:38 AM, Dibyendu Bhattacharya < dibyendu.bhattach...@gmail.com> wrote: > Just to follow up this thread further . > > I was doing some fault tolerant testi

Re: Spark Streaming - Design considerations/Knobs

2015-05-24 Thread Tathagata Das
it? > > Thanks, > Hemant > > On Thu, May 21, 2015 at 2:21 AM, Tathagata Das > wrote: > >> Correcting the ones that are incorrect or incomplete. BUT this is good >> list for things to remember about Spark Streaming. >> >> >> On Wed, May 20, 2015 at 3

Re: StreamingContextSuite fails with NoSuchMethodError

2015-05-30 Thread Tathagata Das
Did was it a clean compilation? TD On Fri, May 29, 2015 at 10:48 PM, Ted Yu wrote: > Hi, > I ran the following command on 1.4.0 RC3: > > mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive package > > I saw the following failure: > > ^[[32mStreamingContextSuite:^[[0m > ^[[32m- from no conf co

Re: [VOTE] Release Apache Spark 1.4.0 (RC4)

2015-06-07 Thread Tathagata Das
+1 On Sun, Jun 7, 2015 at 3:01 PM, Joseph Bradley wrote: > +1 > > On Sat, Jun 6, 2015 at 7:55 PM, Guoqiang Li wrote: > >> +1 (non-binding) >> >> >> -- Original -- >> *From: * "Reynold Xin";; >> *Date: * Fri, Jun 5, 2015 03:18 PM >> *To: * "Krishna Sankar"; >> *Cc

Re: Calculating tuple count /input rate with time

2015-06-23 Thread Tathagata Das
This should give accurate count for each batch, though for getting the rate you have to make sure that you streaming app is stable, that is, batches are processed as fast as they are received (scheduling delay in the spark streaming UI is approx 0). TD On Tue, Jun 23, 2015 at 2:49 AM, anshu shukl

Re: Time is ugly in Spark Streaming....

2015-06-27 Thread Tathagata Das
Could you print the "time" on the driver (that is, in foreachRDD but before RDD.foreachPartition) and see if it is behaving weird? TD On Fri, Jun 26, 2015 at 3:57 PM, Emrehan Tüzün wrote: > > > > > On Fri, Jun 26, 2015 at 12:30 PM, Sea <261810...@qq.com> wrote: > >> Hi, all >> >> I find a probl

Re: [VOTE] Release Apache Spark 1.4.1

2015-06-29 Thread Tathagata Das
@Ted, could you elaborate more on what was the test command that you ran? What profiles, using SBT or Maven? TD On Sun, Jun 28, 2015 at 12:21 PM, Patrick Wendell wrote: > Hey Krishna - this is still the current release candidate. > > - Patrick > > On Sun, Jun 28, 2015 at 12:14 PM, Krishna Sanka

Re: [VOTE] Release Apache Spark 1.4.1

2015-06-29 Thread Tathagata Das
@Ted, I ran the following two commands. mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive -DskipTests clean package mvn -Phadoop-2.4 -Dhadoop.version=2.7.0 -Pyarn -Phive -DwildcardSuites=org.apache.spark.streaming.StreamingContextSuite test Using Java version "1.7.0_51", the tests passed norm

Re: [VOTE] Release Apache Spark 1.4.1

2015-07-01 Thread Tathagata Das
+1 On Tue, Jun 30, 2015 at 8:12 PM, Bobby Chowdary wrote: > +1 Tested on CentOS 7 > On Jun 30, 2015 19:38, "Joseph Bradley" wrote: > >> +1 >> >> On Tue, Jun 30, 2015 at 5:27 PM, Reynold Xin wrote: >> >>> +1 >>> >>> On Tue, Jun 23, 2015 at 10:37 PM, Patrick Wendell >>> wrote: >>> Please v

Re: Does RDD checkpointing store the entire state in HDFS?

2015-07-14 Thread Tathagata Das
1. When you set ssc.checkpoint(checkpointDir), the spark streaming periodically saves the state RDD (which is a snapshot of all the state data) to HDFS using RDD checkpointing. In fact, a streaming app with updateStateByKey will not start until you set checkpoint directory. 2. The updateStateByKey

Re: Does RDD checkpointing store the entire state in HDFS?

2015-07-14 Thread Tathagata Das
BTW, this is more like a user-list kind of mail, than a dev-list. The dev-list is for Spark developers. On Tue, Jul 14, 2015 at 4:23 PM, Tathagata Das wrote: > 1. When you set ssc.checkpoint(checkpointDir), the spark streaming > periodically saves the state RDD (which is a snapshot of a

Re: KinesisStreamSuite failing in master branch

2015-07-19 Thread Tathagata Das
I am taking care of this right now. On Sun, Jul 19, 2015 at 6:08 PM, Patrick Wendell wrote: > I think we should just revert this patch on all affected branches. No > reason to leave the builds broken until a fix is in place. > > - Patrick > > On Sun, Jul 19, 2015 at 6:03 PM, Josh Rosen wrote: >

Re: KinesisStreamSuite failing in master branch

2015-07-19 Thread Tathagata Das
The PR to fix this is out. https://github.com/apache/spark/pull/7519 On Sun, Jul 19, 2015 at 6:41 PM, Tathagata Das wrote: > I am taking care of this right now. > > On Sun, Jul 19, 2015 at 6:08 PM, Patrick Wendell > wrote: > >> I think we should just revert this patch on

Re: jenkins failing on Kinesis shard limits

2015-07-25 Thread Tathagata Das
I have remove the flag in the PullRequestBuilder that enabled Kinesis tests. All the Kinesis tests should be ignored now. In the mean time we can fix the tests. On Sat, Jul 25, 2015 at 8:44 AM, Shixiong Zhu wrote: > The issue is in KinesisBackedBlockRDDSuite > > I have sent https://github.com/ap

Re: Writing streaming data to cassandra creates duplicates

2015-07-27 Thread Tathagata Das
You have to partition that data on the Spark Streaming by the primary key, and then make sure insert data into Cassandra atomically per key, or per set of keys in the partition. You can use the combination of the (batch time, and partition Id) of the RDD inside foreachRDD as the unique id for the d

Re: Checkpoint directory structure

2015-09-23 Thread Tathagata Das
Could you provide the logs on when and how you are seeing this error? On Wed, Sep 23, 2015 at 6:32 PM, Bin Wang wrote: > BTW, I just kill the application and restart it. Then the application > cannot recover from checkpoint because of some lost of RDD. So I'm wonder, > if there are some failure

Re: Checkpoint directory structure

2015-09-24 Thread Tathagata Das
ationMaster$$anon$2.run(ApplicationMaster.scala:525) > 15/09/23 17:47:39 INFO yarn.ApplicationMaster: Final app status: FAILED, > exitCode: 15, (reason: User class threw exception: > java.lang.IllegalArgumentException: requirement failed: Checkpoint > directory does not exist: hdfs://

Re: Dynamic DAG use-case for spark streaming.

2015-09-29 Thread Tathagata Das
A very basic support that is there in DStream is DStream.transform() which take arbitrary RDD => RDD function. This function can actually choose to do different computation with time. That may be of help to you. On Tue, Sep 29, 2015 at 12:06 PM, Archit Thakur wrote: > Hi, > > We are using spark

Re: failure notice

2015-10-05 Thread Tathagata Das
What happens when a whole node running your " per node streaming engine (built-in checkpoint and recovery)" fails? Can its checkpoint and recovery mechanism handle whole node failure? Can you recover from the checkpoint on a different node? Spark and Spark Streaming were designed with the idea th

Re: failure notice

2015-10-06 Thread Tathagata Das
rs out duplicate events based on checkpointed egress offset (at most > once semantic) > > hope it makes sense. > > On Mon, Oct 5, 2015 at 3:11 PM, Tathagata Das wrote: > >> What happens when a whole node running your " per node streaming engine >> (built-in checkpoint

Re: Problem building Spark

2015-10-19 Thread Tathagata Das
Seems to be a heap space issue for Maven. Have you configured Maven's memory according the instruction on the web page? export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" On Mon, Oct 19, 2015 at 6:59 PM, Annabel Melongo < melongo_anna...@yahoo.com.invalid> wrote: > I

Re: Spark-1.6.0-preview2 trackStateByKey exception restoring state

2015-11-23 Thread Tathagata Das
My intention is to make it compatible! Filed this bug - https://issues.apache.org/jira/browse/SPARK-11932 Looking into it right now. Thanks for testing it out and reporting this! On Mon, Nov 23, 2015 at 7:22 AM, jan wrote: > Hi guys, > > I'm trying out the new trackStateByKey API of the Spark-1

Re: UpdateStateByKey : Partitioning and Shuffle

2016-01-05 Thread Tathagata Das
Both mapWithState and updateStateByKey by default uses the HashPartitioner, and hashes the key in the key-value DStream on which the state operation is applied. The new data and state is partition in the exact same partitioner, so that same keys from the new data (from the input DStream) get shuffl

Re: Structured streaming use of DataFrame vs Datasource

2016-06-16 Thread Tathagata Das
DataFrame is a type alias of Dataset[Row], so externally it seems like Dataset is the main type and DataFrame is a derivative type. However, internally, since everything is processed as Rows, everything uses DataFrames, Type classes used in a Dataset is internally converted to rows for processing.

Re: Structured streaming use of DataFrame vs Datasource

2016-06-16 Thread Tathagata Das
ion? > > For a concrete example, Source.getBatch seems to be a public > interface, but returns DataFrame. > > On Thu, Jun 16, 2016 at 1:42 PM, Tathagata Das > wrote: > > DataFrame is a type alias of Dataset[Row], so externally it seems like > > Dataset is the main t

Re: Structured streaming use of DataFrame vs Datasource

2016-06-16 Thread Tathagata Das
hen that's intentional or not. > > > On Thu, Jun 16, 2016 at 2:50 PM, Tathagata Das > wrote: > > There are different ways to view this. If its confusing to think that > Source > > API returning DataFrames, its equivalent to thinking that you are > returning &

Spark 0.9.1 release

2014-03-19 Thread Tathagata Das
Hello everyone, Since the release of Spark 0.9, we have received a number of important bug fixes and we would like to make a bug-fix release of Spark 0.9.1. We are going to cut a release candidate soon and we would love it if people test it out. We have backported several bug fixes into the 0.9 a

Re: Spark 0.9.1 release

2014-03-19 Thread Tathagata Das
least the part to unpersist broadcast variables > explicitly would be great. > Currently we are running with a custom impl which does something > similar, and I would like to move to standard distribution for that. > > > Thanks, > Mridul > > > On Wed, Mar 19, 2014

Re: Spark 0.9.1 release

2014-03-24 Thread Tathagata Das
> doesn't have permissions to but the submitting user does. > >> > View on spark-project.atlassian.net Preview by Yahoo > >> > > >> > > >> > > >> > > >> > > >> > On Thursday, March 20, 2014 1:35 PM, Bhaska

Re: Spark 0.9.1 release

2014-03-24 Thread Tathagata Das
Patrick, that is a good point. On Mon, Mar 24, 2014 at 12:14 AM, Patrick Wendell wrote: > > Spark's dependency graph in a maintenance > *Modifying* Spark's dependency graph... >

Re: Spark 0.9.1 release

2014-03-24 Thread Tathagata Das
4 1:35 PM, Bhaskar Dutta >> wrote: >> It will be great if >> "SPARK-1101<https://spark-project.atlassian.net/browse/SPARK-1101>: >> Umbrella >> for hardening Spark on YARN" can get into 0.9.1. >> >> Thanks, >> Bhaskar >> >

Re: Spark 0.9.1 release

2014-03-24 Thread Tathagata Das
Maven Central.) > > Thanks > Kevin Markey > > > > > On 03/19/2014 06:07 PM, Tathagata Das wrote: >> >> Hello everyone, >> >> Since the release of Spark 0.9, we have received a number of important bug >> fixes and we would like to make a bug-fix

Re: Spark 0.9.1 release

2014-03-25 Thread Tathagata Das
M, it would have no > negative impact on us. Only a positive impact. > > I just wish that all users of ASM would read FAQ entry 15!!! > > Thanks > Kevin > > > > On 03/24/2014 06:30 PM, Tathagata Das wrote: > >> Hello Kevin, >> >> A fix for SPARK-782 wo

Re: Spark 0.9.1 release

2014-03-25 Thread Tathagata Das
; > Thanks, > Mridul > > On Thu, Mar 20, 2014 at 5:37 AM, Tathagata Das > wrote: > > Hello everyone, > > > > Since the release of Spark 0.9, we have received a number of important > bug > > fixes and we would like to make a bug-fix release of Spark 0.9.1.

Re: [VOTE] Release Apache Spark 0.9.1 (rc1)

2014-03-26 Thread Tathagata Das
CDH4 versions and > looked at the new streaming docs. > > > > The release notes seem slightly incomplete, but I guess you're still > working on them? Anyway those don't go into the release tarball so it's > okay. > > > > Matei > > > > O

Re: Spark 0.9.1 release

2014-03-26 Thread Tathagata Das
> - Patrick > > > On Tue, Mar 25, 2014 at 10:47 PM, Mridul Muralidharan >wrote: > > > On Wed, Mar 26, 2014 at 10:53 AM, Tathagata Das > > wrote: > > > PR 159 seems like a fairly big patch to me. And quite recent, so its > > impact > > > on the sch

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

2014-03-27 Thread Tathagata Das
2 TD On Wed, Mar 26, 2014 at 4:03 AM, Tathagata Das wrote: > Please vote on releasing the following candidate as Apache Spark version > 0.9.1 > > A draft of the release notes along with the CHANGES.txt file is > attached to this e-mail. > > The tag to be voted on is v0.9.1-r

Re: Spark 0.9.1 release

2014-03-27 Thread Tathagata Das
owse/SPARK-1322 Please vote on this candidate on the voting thread. Thanks! TD On Wed, Mar 26, 2014 at 3:09 PM, Tathagata Das wrote: > Updates: > 1. Fix for the ASM problem that Kevin mentioned is already in Spark 0.9.1 > RC2 > 2. Fix for pyspark's RDD.top() that Patrick men

Re: [VOTE] Release Apache Spark 0.9.1 (RC3)

2014-03-29 Thread Tathagata Das
Small fixes to the docs can be done after the voting has completed. This should not determine the vote on the release candidate binaries. Please vote as "+1" if the published artifacts and binaries are good to go. TD On Mar 29, 2014 5:23 AM, "prabeesh k" wrote: > https://github.com/apache/spark/

Re: [VOTE] Release Apache Spark 0.9.1 (RC3)

2014-03-29 Thread Tathagata Das
e necessary to update the same after the voting has completed? On Sat, Mar 29, 2014 at 9:28 PM, Tathagata Das wrote: > Small fixes to the docs can be done after the voting has completed. This > should not determine the vote on the release candidate binaries. Please > vote as "+1&q

Re: [VOTE] Release Apache Spark 0.9.1 (RC3)

2014-03-31 Thread Tathagata Das
untu12.04 64bit > > > > > > > > > On Mon, Mar 31, 2014 at 3:56 AM, Matei Zaharia < > matei.zaha...@gmail.com > > > >wrote: > > > > > > > +1 tested on Mac OS X. > > > > > > > > Matei > > > > > > &g

Re: Flaky streaming tests

2014-04-07 Thread Tathagata Das
Yes, I will take a look at those tests ASAP. TD On Mon, Apr 7, 2014 at 11:32 AM, Patrick Wendell wrote: > TD - do you know what is going on here? > > I looked into this ab it and at least a few of these that use > Thread.sleep() and assume the sleep will be exact, which is wrong. We > should

Spark 0.9.1 released

2014-04-09 Thread Tathagata Das
, Matei Zaharia, Nan Zhu, Nick Lanham, Patrick Wendell, Prabin Banka, Prashant Sharma, Qiuzhuang, Raymond Liu, Reynold Xin, Sandy Ryza, Sean Owen, Shixiong Zhu, shiyun.wxm, Stevo Slavić, Tathagata Das, Tom Graves, Xiangrui Meng TD

Re: Spark 0.9.1 released

2014-04-09 Thread Tathagata Das
A small additional note: Please use the direct download links in the Spark Downloads <http://spark.apache.org/downloads.html> page. The Apache mirrors take a day or so to sync from the main repo, so may not work immediately. TD On Wed, Apr 9, 2014 at 2:54 PM, Tathagata Das wrote: > Hi

Re: Spark 0.9.1 released

2014-04-09 Thread Tathagata Das
ark/index.html>hasn't > been updated yet to reflect the new additions to PySpark. > > Nick > > > > On Wed, Apr 9, 2014 at 6:07 PM, Matei Zaharia wrote: > >> Thanks TD for managing this release, and thanks to everyone who >> contributed! >> >> M

Re: Double lhbase dependency in spark 0.9.1

2014-04-17 Thread Tathagata Das
Aaah, this should have been ported to Spark 0.9.1! TD On Thu, Apr 17, 2014 at 12:08 PM, Sean Owen wrote: > I remember that too, and it has been fixed already in master, but > maybe it was not included in 0.9.1: > > https://github.com/apache/spark/blob/master/project/SparkBuild.scala#L367 > --

[VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-20 Thread Tathagata Das
Please vote on releasing the following candidate as Apache Spark version 1.0.0! This has a few bug fixes on top of rc9: SPARK-1875: https://github.com/apache/spark/pull/824 SPARK-1876: https://github.com/apache/spark/pull/819 SPARK-1878: https://github.com/apache/spark/pull/822 SPARK-1879: https:/

Re: [VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-22 Thread Tathagata Das
Hey all, On further testing, I came across a bug that breaks execution of pyspark scripts on YARN. https://issues.apache.org/jira/browse/SPARK-1900 This is a blocker and worth cutting a new RC. We also found a fix for a known issue that prevents additional jar files to be specified through spark-

Re: [VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-22 Thread Tathagata Das
p people follow the active VOTE > threads? The VOTE emails are getting a bit hard to follow. > > > - Henry > > > On Thu, May 22, 2014 at 2:05 PM, Tathagata Das > wrote: >> Hey all, >> >> On further testing, I came across a bug that breaks execution of >

[CANCEL][VOTE] Release Apache Spark 1.0.0 (RC10)

2014-05-22 Thread Tathagata Das
Hey all, We are canceling the vote on RC10 because of a blocker bug in pyspark on Yarn. https://issues.apache.org/jira/browse/SPARK-1900 Thanks everyone for testing! We will post RC11 soon. TD

Re: No output from Spark Streaming program with Spark 1.0

2014-05-23 Thread Tathagata Das
Few more suggestions. 1. See the web ui, is the system running any jobs? If not, then you may need to give the system more nodes. Basically the system should have more cores than the number of receivers. 2. Furthermore there is a streaming specific web ui which gives more streaming specific data.

Re: No output from Spark Streaming program with Spark 1.0

2014-05-24 Thread Tathagata Das
s and I’m seeing the same thing. The > Kafka sources are alive and well and the programs all worked on 0.9 from > Eclipse. And there’s no indication of any failure — just no records are > being delivered. > > Any ideas would be much appreciated … > > > Thanks, > >

[VOTE] Release Apache Spark 1.0.0 (RC11)

2014-05-26 Thread Tathagata Das
Please vote on releasing the following candidate as Apache Spark version 1.0.0! This has a few important bug fixes on top of rc10: SPARK-1900 and SPARK-1918: https://github.com/apache/spark/pull/853 SPARK-1870: https://github.com/apache/spark/pull/848 SPARK-1897: https://github.com/apache/spark/pu

[RESULT][VOTE] Release Apache Spark 1.0.0 (RC11)

2014-05-29 Thread Tathagata Das
aputra Sean McNamara* Xiangrui Meng* Andy Konwinski* Krishna Sankar Kevin Markey Patrick Wendell* Tathagata Das* 0: (1 vote) Ankur Dave* -1: (0 vote) Please hold off announcing Spark 1.0.0 until Apache Software Foundation makes the press release tomorrow. Thank you very much for your cooperation. TD

Re: [VOTE] Release Apache Spark 1.0.0 (RC11)

2014-05-29 Thread Tathagata Das
n Henry Saputra Sean McNamara* Xiangrui Meng* Andy Konwinski* Krishna Sankar Kevin Markey Patrick Wendell* Tathagata Das* 0: (1 vote) Ankur Dave* -1: (0 vote) * = binding Please hold off announcing Spark 1.0.0 until Apache Software Foundation makes the press release tomorrow. Thank you very much

Re: [brainsotrming] Generalization of DStream, a ContinuousRDD ?

2014-07-15 Thread Tathagata Das
Very interesting ideas Andy! Conceptually i think it makes sense. In fact, it is true that dealing with time series data, windowing over application time, windowing over number of events, are things that DStream does not natively support. The real challenge is actually mapping the conceptual windo

Re: [brainsotrming] Generalization of DStream, a ContinuousRDD ?

2014-07-16 Thread Tathagata Das
lla > about.me/noootsab > [image: aℕdy ℙetrella on about.me] > > <http://about.me/noootsab> > > > On Wed, Jul 16, 2014 at 12:33 AM, Tathagata Das < > tathagata.das1...@gmail.com > > wrote: > > > Very interesting ideas Andy! > > > > Conceptua

Re: Does RDD checkpointing store the entire state in HDFS?

2014-07-16 Thread Tathagata Das
After every checkpointing interval, the latest state RDD is stored to HDFS in its entirety. Along with that, the series of DStream transformations that was setup with the streaming context is also stored into HDFS (the whole DAG of DStream objects is serialized and saved). TD On Wed, Jul 16, 201

Re: Tranforming flume events using Spark transformation functions

2014-07-22 Thread Tathagata Das
This is because of the RDD's lazy evaluation! Unless you force a transformed (mapped/filtered/etc.) RDD to give you back some data (like RDD.count) or output the data (like RDD.saveAsTextFile()), Spark will not do anything. So after the eventData.map(...), if you do take(10) and then print the res

[VOTE] Release Apache Spark 1.0.2 (RC1)

2014-07-25 Thread Tathagata Das
Please vote on releasing the following candidate as Apache Spark version 1.0.2. This release fixes a number of bugs in Spark 1.0.1. Some of the notable ones are - SPARK-2452: Known issue is Spark 1.0.1 caused by attempted fix for SPARK-1199. The fix was reverted for 1.0.2. - SPARK-2576: NoClassDef

Re: [VOTE] Release Apache Spark 1.0.2 (RC1)

2014-07-28 Thread Tathagata Das
Let me add my vote as well. Did some basic tests by running simple projects with various Spark modules. Tested checksums. +1 On Sun, Jul 27, 2014 at 4:52 PM, Matei Zaharia wrote: > +1 > > Tested this on Mac OS X. > > Matei > > On Jul 25, 2014, at 4:08 PM, Tathagata Das

Re: failed to build spark with maven for both 1.0.1 and latest master branch

2014-07-31 Thread Tathagata Das
Does a "mvn clean" or "sbt/sbt clean" help? TD On Wed, Jul 30, 2014 at 9:25 PM, yao wrote: > Hi Folks, > > Today I am trying to build spark using maven; however, the following > command failed consistently for both 1.0.1 and the latest master. (BTW, it > seems sbt works fine: *sbt/sbt -Dhadoop.

Re: Low Level Kafka Consumer for Spark

2014-08-06 Thread Tathagata Das
Hi Dibyendu, This is really awesome. I am still yet to go through the code to understand the details, but I want to do it really soon. In particular, I want to understand the improvements, over the existing Kafka receiver. And its fantastic to see such contributions from the community. :) TD On

Re: reference to dstream in package org.apache.spark.streaming which is not available

2014-08-22 Thread Tathagata Das
Figured it out. Fixing this ASAP. TD On Fri, Aug 22, 2014 at 5:51 PM, Patrick Wendell wrote: > Hey All, > > We can sort this out ASAP. Many of the Spark committers were at a company > offsite for the last 72 hours, so sorry that it is broken. > > - Patrick > > > On Fri, Aug 22, 2014 at 4:07 PM

Re: reference to dstream in package org.apache.spark.streaming which is not available

2014-08-22 Thread Tathagata Das
The real fix is that the spark sink suite does not really need to use to the spark-streaming test jars. Removing that dependency altogether, and submitting a PR. TD On Fri, Aug 22, 2014 at 6:34 PM, Tathagata Das wrote: > Figured it out. Fixing this ASAP. > > TD > > > On Fri,

  1   2   >