Re: question on Write Ahead Log (Spark Streaming )

2017-03-10 Thread Dibyendu Bhattacharya
Hi, You could also use this Receiver : https://github.com/dibbhatt/kafka-spark-consumer This is part of spark-packages also : https://spark-packages.org/package/dibbhatt/kafka-spark-consumer You do not need to enable WAL in this and still recover from Driver failure with no data loss. You can

Re: question on Write Ahead Log (Spark Streaming )

2017-03-08 Thread Saisai Shao
IIUC, your scenario is quite like what currently ReliableKafkaReceiver does. You can only send ack to the upstream source after WAL is persistent, otherwise because of asynchronization of data processing and data receiving, there's still a chance data could be lost if you send out ack before WAL.

question on Write Ahead Log (Spark Streaming )

2017-03-08 Thread kant kodali
Hi All, I am using a Receiver based approach. And I understand that spark streaming API's will convert the received data from receiver into blocks and these blocks that are in memory are also stored in WAL if one enables it. my upstream source which is not Kafka can also replay by which I mean if

Re: Write Ahead Log

2016-06-08 Thread Mohit Anchlia
Is there any specific reason why this feature is only supported in streaming? On Wed, Jun 8, 2016 at 3:24 PM, Ted Yu wrote: > There was a minor typo in the name of the config: > > spark.streaming.receiver.writeAheadLog.enable > > Yes, it only applies to Streaming. > > On

Re: Write Ahead Log

2016-06-08 Thread Ted Yu
There was a minor typo in the name of the config: spark.streaming.receiver.writeAheadLog.enable Yes, it only applies to Streaming. On Wed, Jun 8, 2016 at 3:14 PM, Mohit Anchlia wrote: > Is something similar to park.streaming.receiver.writeAheadLog.enable > available on

Write Ahead Log

2016-06-08 Thread Mohit Anchlia
Is something similar to park.streaming.receiver.writeAheadLog.enable available on SparkContext? It looks like it only works for spark streaming.

Re: Spark streaming - update configuration while retaining write ahead log data?

2016-03-15 Thread Ted Yu
ation. > > Thanks > Saisai > >> On Tue, Mar 15, 2016 at 5:12 PM, Ewan Leith <ewan.le...@realitymine.com> >> wrote: >> Has anyone seen a way of updating the Spark streaming job configuration >> while retaining the existing data in the write ahead log? &

RE: Spark streaming - update configuration while retaining write ahead log data?

2016-03-15 Thread Ewan Leith
That’s what I thought, it’s a shame! Thanks Saisai, Ewan From: Saisai Shao [mailto:sai.sai.s...@gmail.com] Sent: 15 March 2016 09:22 To: Ewan Leith <ewan.le...@realitymine.com> Cc: user <user@spark.apache.org> Subject: Re: Spark streaming - update configuration while retaining writ

Re: Spark streaming - update configuration while retaining write ahead log data?

2016-03-15 Thread Saisai Shao
no way to handle your situation. Thanks Saisai On Tue, Mar 15, 2016 at 5:12 PM, Ewan Leith <ewan.le...@realitymine.com> wrote: > Has anyone seen a way of updating the Spark streaming job configuration > while retaining the existing data in the write ahead log? > > > >

Spark streaming - update configuration while retaining write ahead log data?

2016-03-15 Thread Ewan Leith
Has anyone seen a way of updating the Spark streaming job configuration while retaining the existing data in the write ahead log? e.g. if you've launched a job without enough executors and a backlog has built up in the WAL, can you increase the number of executors without losing the WAL data

can checkpoint and write ahead log save the data in queued batch?

2016-03-11 Thread Yu Xie
Hi spark user I am running an spark streaming app that use receiver from a pubsub system, and the pubsub system does NOT support ack. And I don't want the data to be lost if there is a driver failure, and by accident, the batches queue up at that time. I tested by generating some queued

Re: Spark Streaming Write Ahead Log (WAL) not replaying data after restart

2016-01-26 Thread Patrick McGloin
ing-programming-guide.html#how-to-configure-checkpointing > > On Thu, Jan 21, 2016 at 3:32 AM, Patrick McGloin < > mcgloin.patr...@gmail.com> wrote: > >> Hi all, >> >> To have a simple way of testing the Spark Streaming Write Ahead Log I >> created a very

Re: Spark Streaming Write Ahead Log (WAL) not replaying data after restart

2016-01-25 Thread Shixiong(Ryan) Zhu
t; Hi all, > > To have a simple way of testing the Spark Streaming Write Ahead Log I > created a very simple Custom Input Receiver, which will generate strings > and store those: > > class InMemoryStringReceiver extends > Receiver[String](StorageLevel.MEMORY_AND

Spark Streaming Write Ahead Log (WAL) not replaying data after restart

2016-01-21 Thread Patrick McGloin
Hi all, To have a simple way of testing the Spark Streaming Write Ahead Log I created a very simple Custom Input Receiver, which will generate strings and store those: class InMemoryStringReceiver extends Receiver[String](StorageLevel.MEMORY_AND_DISK_SER) { val batchID

Does spark streaming write ahead log writes all received data to HDFS ?

2015-11-20 Thread kali.tumm...@gmail.com
folder ? what actually does write ahead log folder has ? Thanks Sri -- View this message in context: http://apache-spark-user-list.1001560.n3.nabble.com/Does-spark-streaming-write-ahead-log-writes-all-received-data-to-HDFS-tp25439.html Sent from the Apache Spark User List mailing list