Re: Re: spark+kafka+dynamic resource allocation

2023-01-30 Thread Mich Talebzadeh
t; Lingzhe Sun > > > *From:* Mich Talebzadeh > *Date:* 2023-01-30 02:14 > *To:* Lingzhe Sun > *CC:* ashok34...@yahoo.com; User > *Subject:* Re: Re: spark+kafka+dynamic resource allocation > Hi, > > Spark Structured Streaming currently does not support dynamic allocation

Re: Re: spark+kafka+dynamic resource allocation

2023-01-29 Thread Lingzhe Sun
: ashok34...@yahoo.com; User Subject: Re: Re: spark+kafka+dynamic resource allocation Hi, Spark Structured Streaming currently does not support dynamic allocation (see SPARK-24815: Structured Streaming should support dynamic allocation). which is still open Autoscaling in Cloud offerings like Google

Re: Re: spark+kafka+dynamic resource allocation

2023-01-29 Thread Mich Talebzadeh
* 2023-01-29 04:01 > *To:* User ; Lingzhe Sun > *Subject:* Re: spark+kafka+dynamic resource allocation > Hi, > > Worth checking this link > > > https://spark.apache.org/docs/latest/job-scheduling.html#dynamic-resource-allocation > > On Saturday, 28 January 2023 at 06:18:2

Re: Re: spark+kafka+dynamic resource allocation

2023-01-28 Thread Lingzhe Sun
Thank you for the response. But the reference does not seem to be answering any of those questions. BS Lingzhe Sun From: ashok34...@yahoo.com Date: 2023-01-29 04:01 To: User; Lingzhe Sun Subject: Re: spark+kafka+dynamic resource allocation Hi, Worth checking this link https

Re: spark+kafka+dynamic resource allocation

2023-01-28 Thread ashok34...@yahoo.com.INVALID
Hi, Worth checking this link https://spark.apache.org/docs/latest/job-scheduling.html#dynamic-resource-allocation On Saturday, 28 January 2023 at 06:18:28 GMT, Lingzhe Sun wrote: #yiv9684413148 body {line-height:1.5;}#yiv9684413148 ol, #yiv9684413148 ul

RE: Spark kafka structured streaming - how to prevent dataloss

2022-03-22 Thread Gnanasoundari Soundarajan
Hi all, Any suggestion? Regards, Gnana From: Gnanasoundari Soundarajan Sent: Tuesday, March 8, 2022 10:02 PM To: user@spark.apache.org Subject: Spark kafka structured streaming - how to prevent dataloss Hi, In spark, it uses checkpoints to keep track of offsets in kafka. If there is any

RE: Spark Kafka Integration

2022-02-25 Thread Michael Williams (SSI)
Ahh, ok. So, Kafka 3.1 is supported for Spark 3.2.1. Thank you very much. From: Mich Talebzadeh [mailto:mich.talebza...@gmail.com] Sent: Friday, February 25, 2022 2:50 PM To: Michael Williams (SSI) Cc: user@spark.apache.org Subject: Re: Spark Kafka Integration these are the old and news ones

RE: Spark Kafka Integration

2022-02-25 Thread Michael Williams (SSI)
Thank you, that is good to know. From: Sean Owen [mailto:sro...@gmail.com] Sent: Friday, February 25, 2022 2:46 PM To: Michael Williams (SSI) Cc: Mich Talebzadeh ; user@spark.apache.org Subject: Re: Spark Kafka Integration Spark 3.2.1 is compiled vs Kafka 2.8.0; the forthcoming Spark 3.3

Re: Spark Kafka Integration

2022-02-25 Thread Mich Talebzadeh
ould be appreciated. Our entire team is > totally new to spark and kafka (this is a poc trial). > > > > *From:* Mich Talebzadeh [mailto:mich.talebza...@gmail.com] > *Sent:* Friday, February 25, 2022 2:30 PM > *To:* Michael Williams (SSI) > *Cc:* user@spark.apache.org > *Subjec

Re: Spark Kafka Integration

2022-02-25 Thread Sean Owen
hael Williams (SSI) > *Cc:* user@spark.apache.org > *Subject:* Re: Spark Kafka Integration > > > > and what version of kafka do you have 2.7? > > > > for spark 3.1.1 I needed these jar files to make it work > > > > kafka-clients-2.7.0.jar > commons-pool2-2.

RE: Spark Kafka Integration

2022-02-25 Thread Michael Williams (SSI)
Thank you From: Mich Talebzadeh [mailto:mich.talebza...@gmail.com] Sent: Friday, February 25, 2022 2:35 PM To: Michael Williams (SSI) Cc: Sean Owen ; user@spark.apache.org Subject: Re: Spark Kafka Integration please see my earlier reply for 3.1.1 tested and worked in Google Dataproc

RE: Spark Kafka Integration

2022-02-25 Thread Michael Williams (SSI)
To: Michael Williams (SSI) Cc: user@spark.apache.org Subject: Re: Spark Kafka Integration and what version of kafka do you have 2.7? for spark 3.1.1 I needed these jar files to make it work kafka-clients-2.7.0.jar commons-pool2-2.9.0.jar spark-streaming_2.12-3.1.1.jar spark-sql-kafka-0-10_2.12

Re: Spark Kafka Integration

2022-02-25 Thread Mich Talebzadeh
t the > dependencies already exist on disk. If that makes any sense. > > > > Thank you > > > > *From:* Mich Talebzadeh [mailto:mich.talebza...@gmail.com] > *Sent:* Friday, February 25, 2022 2:16 PM > *To:* Michael Williams (SSI) > *Cc:* user@spark.apache.org > *Subj

Re: Spark Kafka Integration

2022-02-25 Thread Mich Talebzadeh
and what version of kafka do you have 2.7? for spark 3.1.1 I needed these jar files to make it work kafka-clients-2.7.0.jar commons-pool2-2.9.0.jar spark-streaming_2.12-3.1.1.jar spark-sql-kafka-0-10_2.12-3.1.0.jar HTH view my Linkedin profile

RE: Spark Kafka Integration

2022-02-25 Thread Michael Williams (SSI)
exist on disk. If that makes any sense. Thank you From: Mich Talebzadeh [mailto:mich.talebza...@gmail.com] Sent: Friday, February 25, 2022 2:16 PM To: Michael Williams (SSI) Cc: user@spark.apache.org Subject: Re: Spark Kafka Integration What is the use case? Is this for spark structured

Re: Spark Kafka Integration

2022-02-25 Thread Sean Owen
That .jar is available on Maven, though typically you depend on it in your app, and compile an uber JAR which will contain it and all its dependencies. You can I suppose manage to compile an uber JAR from that dependency itself with tools if needed. On Fri, Feb 25, 2022 at 1:37 PM Michael

Re: Spark Kafka Integration

2022-02-25 Thread Mich Talebzadeh
What is the use case? Is this for spark structured streaming? HTH view my Linkedin profile https://en.everybodywiki.com/Mich_Talebzadeh *Disclaimer:* Use it at your own risk. Any and all responsibility for any loss, damage or

Re: Spark Kafka Streaming With Transactional Messages

2020-09-16 Thread jianyangusa
I have the same issue. Do you have a solution? Maybe spark stream not support transaction message. I use Kafka stream to retrieve the transaction message. Maybe we can ask Spark support this feature. -- Sent from: http://apache-spark-user-list.1001560.n3.nabble.com/

Re: [Spark Kafka Structured Streaming] Adding partition and topic to the kafka dynamically

2020-08-28 Thread Amit Joshi
Hi Jungtaek, Thanks for the input. I did tried and it worked. I got confused earlier after reading some blogs. Regards Amit On Friday, August 28, 2020, Jungtaek Lim wrote: > Hi Amit, > > if I remember correctly, you don't need to restart the query to reflect > the newly added topic and

Re: [Spark Kafka Structured Streaming] Adding partition and topic to the kafka dynamically

2020-08-28 Thread Gabor Somogyi
Hi Amit, The answer is no. G On Fri, Aug 28, 2020 at 9:16 AM Jungtaek Lim wrote: > Hi Amit, > > if I remember correctly, you don't need to restart the query to reflect > the newly added topic and partition, if your subscription covers the topic > (like subscribe pattern). Please try it out.

Re: [Spark Kafka Structured Streaming] Adding partition and topic to the kafka dynamically

2020-08-28 Thread Jungtaek Lim
Hi Amit, if I remember correctly, you don't need to restart the query to reflect the newly added topic and partition, if your subscription covers the topic (like subscribe pattern). Please try it out. Hope this helps. Thanks, Jungtaek Lim (HeartSaVioR) On Fri, Aug 28, 2020 at 1:56 PM Amit

Re: [Spark Kafka Structured Streaming] Adding partition and topic to the kafka dynamically

2020-08-27 Thread Amit Joshi
Any pointers will be appreciated. On Thursday, August 27, 2020, Amit Joshi wrote: > Hi All, > > I am trying to understand the effect of adding topics and partitions to a > topic in kafka, which is being consumed by spark structured streaming > applications. > > Do we have to restart the spark

Re: Spark Kafka Streaming with Offset Gaps

2020-05-21 Thread nimmi.cv
Did you resolve this issue ? -- Sent from: http://apache-spark-user-list.1001560.n3.nabble.com/ - To unsubscribe e-mail: user-unsubscr...@spark.apache.org

Re: Spark Kafka Streaming making progress but there is no data to be consumed

2019-09-11 Thread Charles vinodh
Thanks Dhaval, that fixed the issue. The constant resetting of Kafka offsets misled me about the issue. Please feel free the answer the SO question here if you would like to..

Re: Spark Kafka Streaming making progress but there is no data to be consumed

2019-09-11 Thread Dhaval Patel
Hi Charles, Can you check is any of the case related to output directory and checkpoint location mentioned in below link is applicable in your case? https://kb.databricks.com/streaming/file-sink-streaming.html Regards Dhaval On Wed, Sep 11, 2019 at 9:29 PM Burak Yavuz wrote: > Hey Charles, >

Re: Spark Kafka Streaming making progress but there is no data to be consumed

2019-09-11 Thread Burak Yavuz
Hey Charles, If you are using maxOffsetsPerTrigger, you will likely rest the offsets every microbatch, because: 1. Spark will figure out a range of offsets to process (let's call them x and y) 2. If these offsets have fallen out of the retention period, Spark will try to set the offset to x

Re: Spark Kafka Streaming making progress but there is no data to be consumed

2019-09-11 Thread Charles vinodh
Hi Sandish, as I have said if the offset reset happens only once that would make sense. But I am not sure how to explain why the offset reset is happening for every micro-batch... ideally once the offset reset happens the app should move to a valid offset and start consuming data. but in my case

Re: Spark Kafka Streaming making progress but there is no data to be consumed

2019-09-11 Thread Sandish Kumar HN
You can see this kind of error, if there is consumer lag more than Kafka retention period. You will not see any failures if below option is not set. Set failOnDataLoss=true option to see failures. On Wed, Sep 11, 2019 at 3:24 PM Charles vinodh wrote: > The only form of rate limiting I have set

Re: Spark Kafka Streaming making progress but there is no data to be consumed

2019-09-11 Thread Charles vinodh
The only form of rate limiting I have set is *maxOffsetsPerTrigger *and *fetch.message.max.bytes. * *"*may be that you are trying to process records that have passed the retention period within Kafka.*"* If the above is true then I should have my offsets reset only once ideally when my

Re: Spark Kafka Streaming making progress but there is no data to be consumed

2019-09-11 Thread Burak Yavuz
Do you have rate limiting set on your stream? It may be that you are trying to process records that have passed the retention period within Kafka. On Wed, Sep 11, 2019 at 2:39 PM Charles vinodh wrote: > > Hi, > > I am trying to run a spark application ingesting data from Kafka using the > Spark

Re: Spark kafka streaming job stopped

2019-06-11 Thread Amit Sharma
Please provide update if any one knows. On Monday, June 10, 2019, Amit Sharma wrote: > > We have spark kafka sreaming job running on standalone spark cluster. We > have below kafka architecture > > 1. Two cluster running on two data centers. > 2. There is LTM on top on each data center (load

Re: Spark Kafka Batch Write guarantees

2019-04-01 Thread hemant singh
Thanks Shixiong, read in documentation as well that duplicates might exist because of task retries. On Mon, 1 Apr 2019 at 9:43 PM, Shixiong(Ryan) Zhu wrote: > The Kafka source doesn’t support transaction. You may see partial data or > duplicated data if a Spark task fails. > > On Wed, Mar 27,

Re: Spark Kafka Batch Write guarantees

2019-04-01 Thread Shixiong(Ryan) Zhu
The Kafka source doesn’t support transaction. You may see partial data or duplicated data if a Spark task fails. On Wed, Mar 27, 2019 at 1:15 AM hemant singh wrote: > We are using spark batch to write Dataframe to Kafka topic. The spark > write function with write.format(source = Kafka). > Does

Re: Spark-Kafka integration - build failing with sbt

2017-06-19 Thread Cody Koeninger
org.apache.spark.streaming.kafka.KafkaUtils is in the spark-streaming-kafka-0-8 project On Mon, Jun 19, 2017 at 1:01 PM, karan alang wrote: > Hi Cody - i do have a additional basic question .. > > When i tried to compile the code in Eclipse, i was not able to do that > >

Re: Spark-Kafka integration - build failing with sbt

2017-06-19 Thread karan alang
Hi Cody - i do have a additional basic question .. When i tried to compile the code in Eclipse, i was not able to do that eg. import org.apache.spark.streaming.kafka.KafkaUtils gave errors saying KafaUtils was not part of the package. However, when i used sbt to compile - the compilation went

Re: Spark-Kafka integration - build failing with sbt

2017-06-17 Thread karan alang
Thanks, Cody .. yes, was able to fix that. On Sat, Jun 17, 2017 at 1:18 PM, Cody Koeninger wrote: > There are different projects for different versions of kafka, > spark-streaming-kafka-0-8 and spark-streaming-kafka-0-10 > > See > >

Re: Spark-Kafka integration - build failing with sbt

2017-06-17 Thread Cody Koeninger
There are different projects for different versions of kafka, spark-streaming-kafka-0-8 and spark-streaming-kafka-0-10 See http://spark.apache.org/docs/latest/streaming-kafka-integration.html On Fri, Jun 16, 2017 at 6:51 PM, karan alang wrote: > I'm trying to compile

Re: spark kafka consumer with kerberos

2017-03-31 Thread Saisai Shao
Hi Bill, Normally Kerberos principal and keytab should be enough, because keytab could actually represent the password. Did you configure SASL/GSSAPI or SASL/Plain for KafkaClient? http://kafka.apache.org/documentation.html#security_sasl Actually this is more like a Kafka question and normally

Re: spark kafka consumer with kerberos

2017-03-31 Thread Bill Schwanitz
Saisai, Yea that seems to have helped. Looks like the kerberos ticket when I submit does not get passed to the executor? ... 3 more Caused by: org.apache.kafka.common.KafkaException: javax.security.auth.login.LoginException: Unable to obtain password from user at

Re: spark kafka consumer with kerberos

2017-03-31 Thread Saisai Shao
Hi Bill, The exception is from executor side. From the gist you provided, looks like the issue is that you only configured java options in driver side, I think you should also configure this in executor side. You could refer to here (

Re: [Spark Kafka] API Doc pages for Kafka 0.10 not current

2017-02-28 Thread Cody Koeninger
The kafka-0-8 and kafka-0-10 integrations have conflicting dependencies. Last time I checked, Spark's doc publication puts everything all in one classpath, so publishing them both together won't work. I thought there was already a Jira ticket related to this, but a quick look didn't turn it up.

Re: [Spark Kafka] How to update batch size of input dynamically for spark kafka consumer?

2017-01-03 Thread Cody Koeninger
You can't change the batch time, but you can limit the number of items in the batch http://spark.apache.org/docs/latest/configuration.html spark.streaming.backpressure.enabled spark.streaming.kafka.maxRatePerPartition On Tue, Jan 3, 2017 at 4:00 AM, 周家帅 wrote: > Hi, > > I am

Re: Spark kafka integration issues

2016-09-14 Thread Cody Koeninger
Yeah, an updated version of that blog post is available at https://github.com/koeninger/kafka-exactly-once On Wed, Sep 14, 2016 at 11:35 AM, Mukesh Jha wrote: > Thanks for the reply Cody. > > I found the below article on the same, very helpful. Thanks for the details, >

Re: Spark kafka integration issues

2016-09-14 Thread Mukesh Jha
Thanks for the reply Cody. I found the below article on the same, very helpful. Thanks for the details, much appreciated. http://blog.cloudera.com/blog/2015/03/exactly-once-spark-streaming-from-apache-kafka/ On Tue, Sep 13, 2016 at 8:14 PM, Cody Koeninger wrote: > 1. see

Re: Spark kafka integration issues

2016-09-13 Thread Cody Koeninger
1. see http://spark.apache.org/docs/latest/streaming-kafka-integration.html#approach-2-direct-approach-no-receivers look for HasOffsetRange. If you really want the info per-message rather than per-partition, createRDD has an overload that takes a messageHandler from MessageAndMetadata to

Re: Spark Kafka stream processing time increasing gradually

2016-06-22 Thread Roshan Singh
Thanks for the detailed explanation. Just tested it, worked like a charm. On Mon, Jun 20, 2016 at 1:02 PM, N B wrote: > Its actually necessary to retire keys that become "Zero" or "Empty" so to > speak. In your case, the key is "imageURL" and values are a dictionary, one >

Re: Spark Kafka stream processing time increasing gradually

2016-06-20 Thread N B
Its actually necessary to retire keys that become "Zero" or "Empty" so to speak. In your case, the key is "imageURL" and values are a dictionary, one of whose fields is "count" that you are maintaining. For simplicity and illustration's sake I will assume imageURL to be a strings like "abc". Your

Re: Spark Kafka stream processing time increasing gradually

2016-06-16 Thread Roshan Singh
Hi, According to the docs ( https://spark.apache.org/docs/latest/api/python/pyspark.streaming.html#pyspark.streaming.DStream.reduceByKeyAndWindow), filerFunc can be used to retain expiring keys. I do not want to retain any expiring key, so I do not understand how can this help me stabilize it.

Re: Spark Kafka stream processing time increasing gradually

2016-06-16 Thread N B
We had this same issue with the reduceByKeyAndWindow API that you are using. For fixing this issue, you have to use different flavor of that API, specifically the 2 versions that allow you to give a 'Filter function' to them. Putting in the filter functions helped stabilize our application too.

Re: Spark + Kafka processing trouble

2016-05-31 Thread Malcolm Lockyer
Thanks for the suggestions. I agree that there isn't some magic configuration setting, or that the sql options have some flaw - I just intended to explain the frustration of having a non-trivial (but still simple) Spark streaming job running on tiny amounts of data performing absolutely horribly.

Re: Spark + Kafka processing trouble

2016-05-31 Thread Cody Koeninger
> 500ms is I believe the minimum batch interval for Spark micro batching. It's better to test than to believe, I've run 250ms jobs. Same applies to the comments around JDBC, why assume when you could (dis)prove? It's not like it's a lot of effort to set up a minimal job that does

Re: Spark + Kafka processing trouble

2016-05-31 Thread Mich Talebzadeh
500ms is I believe the minimum batch interval for Spark micro batching. However, a JDBC call is a use of Unix file descriptor and context switch and it does have performance implication. That is irrespective of Kafka as it is happening one is actually going through Hive JDBC. It is a classic

Re: Spark + Kafka processing trouble

2016-05-31 Thread Cody Koeninger
There isn't a magic spark configuration setting that would account for multiple-second-long fixed overheads, you should be looking at maybe 200ms minimum for a streaming batch. 1024 kafka topicpartitions is not reasonable for the volume you're talking about. Unless you have really extreme

Re: Spark + Kafka processing trouble

2016-05-31 Thread Alonso Isidoro Roman
Mich`s idea is quite fine, if i was you, i will follow his idea... Alonso Isidoro Roman [image: https://]about.me/alonso.isidoro.roman 2016-05-31 6:37 GMT+02:00 Mich Talebzadeh

Re: Spark + Kafka processing trouble

2016-05-30 Thread Mich Talebzadeh
how are you getting your data from the database. Are you using JDBC. Can you actually call the database first (assuming the same data, put it in temp table in Spark and cache it for the duration of windows length and use the data from the cached table? Dr Mich Talebzadeh LinkedIn *

Re: Spark + Kafka processing trouble

2016-05-30 Thread Malcolm Lockyer
On Tue, May 31, 2016 at 3:14 PM, Darren Govoni wrote: > Well that could be the problem. A SQL database is essential a big > synchronizer. If you have a lot of spark tasks all bottlenecking on a single > database socket (is the database clustered or colocated with spark

Re: Spark + Kafka processing trouble

2016-05-30 Thread Darren Govoni
from my Verizon Wireless 4G LTE smartphone Original message From: Malcolm Lockyer <malcolm.lock...@hapara.com> Date: 05/30/2016 10:40 PM (GMT-05:00) To: user@spark.apache.org Subject: Re: Spark + Kafka processing trouble On Tue, May 31, 2016 at 1:56 PM, Darren Govon

Re: Spark + Kafka processing trouble

2016-05-30 Thread Malcolm Lockyer
On Tue, May 31, 2016 at 1:56 PM, Darren Govoni wrote: > So you are calling a SQL query (to a single database) within a spark > operation distributed across your workers? Yes, but currently with very small sets of data (1-10,000) and on a single (dev) machine right now.

RE: Spark + Kafka processing trouble

2016-05-30 Thread Darren Govoni
So you are calling a SQL query (to a single database) within a spark operation distributed across your workers?  Sent from my Verizon Wireless 4G LTE smartphone Original message From: Malcolm Lockyer Date: 05/30/2016 9:45 PM (GMT-05:00)

Re: Spark + Kafka all messages being used in 1 batch

2016-03-06 Thread Shahbaz
- Do you happen to see how busy are the nodes in terms of CPU and how much heap each executor is allocated with. - If there is enough capacity ,you may want to increase number of cores per executor to 2 and do the needed heap tweaking. - How much time did it take to process 4M+

Re: Spark + Kafka all messages being used in 1 batch

2016-03-06 Thread Vinti Maheshwari
I have 2 machines in my cluster with the below specifications: 128 GB RAM and 8 cores machine Regards, ~Vinti On Sun, Mar 6, 2016 at 7:54 AM, Vinti Maheshwari wrote: > Thanks Supreeth and Shahbaz. I will try adding > spark.streaming.kafka.maxRatePerPartition. > > Hi

Re: Spark + Kafka all messages being used in 1 batch

2016-03-06 Thread Vinti Maheshwari
Thanks Supreeth and Shahbaz. I will try adding spark.streaming.kafka.maxRatePerPartition. Hi Shahbaz, Please see comments, inline: - Which version of Spark you are using. ==> *1.5.2* - How big is the Kafka Cluster ==> *2 brokers* - What is the Message Size and type.==> *String, 9,550

Re: Spark + Kafka all messages being used in 1 batch

2016-03-05 Thread Supreeth
Try setting spark.streaming.kafka.maxRatePerPartition, this can help control the number of messages read from Kafka per partition on the spark streaming consumer. -S > On Mar 5, 2016, at 10:02 PM, Vinti Maheshwari wrote: > > Hello, > > I am trying to figure out why my

Re: Spark Kafka Direct Error

2015-11-24 Thread swetha kasireddy
Is it possible that the kafka offset api is somehow returning the wrong offsets. Because each time the job fails for different partitions with an error similar to the error that I get below. Job aborted due to stage failure: Task 20 in stage 117.0 failed 4 times, most recent failure: Lost task

Re: Spark Kafka Direct Error

2015-11-24 Thread Cody Koeninger
Anything's possible, but that sounds pretty unlikely to me. Are the partitions it's failing for all on the same leader? Have there been any leader rebalances? Do you have enough log retention? If you log the offset for each message as it's processed, when do you see the problem? On Tue, Nov 24,

Re: Spark Kafka Direct Error

2015-11-24 Thread swetha kasireddy
I see the assertion error when I compare the offset ranges as shown below. How do I log the offset for each message? kafkaStream.transform { rdd => // Get the offset ranges in the RDD offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges rdd }.foreachRDD { rdd => for (o <-

Re: Spark Kafka Direct Error

2015-11-23 Thread swetha kasireddy
Does Kafka direct query the offsets from the zookeeper directly? From where does it get the offsets? There is data in those offsets, but somehow Kafka Direct does not seem to pick it up. Other Consumers that use Zoo Keeper Quorum of that Stream seems to be fine. Only Kafka Direct seems to have

Re: Spark Kafka Direct Error

2015-11-23 Thread Cody Koeninger
No, that means that at the time the batch was scheduled, the kafka leader reported the ending offset was 221572238, but during processing, kafka stopped returning messages before reaching that ending offset. That probably means something got screwed up with Kafka - e.g. you lost a leader and lost

RE: Spark/Kafka Streaming Job Gets Stuck

2015-10-29 Thread Sabarish Sasidharan
m). We're in the process of re-evaluating. > -- > Nick > > -Original Message- > From: Adrian Tanase [mailto:atan...@adobe.com] > Sent: Wednesday, October 28, 2015 4:53 PM > To: Afshartous, Nick <nafshart...@turbine.com> > Cc: user@spark.apache.org > Subject: Re: Sp

Re: Spark/Kafka Streaming Job Gets Stuck

2015-10-29 Thread Adrian Tanase
Original Message- >From: Adrian Tanase [mailto:atan...@adobe.com] >Sent: Wednesday, October 28, 2015 4:53 PM >To: Afshartous, Nick <nafshart...@turbine.com> >Cc: user@spark.apache.org >Subject: Re: Spark/Kafka Streaming Job Gets Stuck > >Does it work as expected with smalle

Re: Spark/Kafka Streaming Job Gets Stuck

2015-10-29 Thread Cody Koeninger
void. That >> was the explanation of the developer who made this decision (who's no >> longer on the team). We're in the process of re-evaluating. >> -- >> Nick >> >> -Original Message- >> From: Adrian Tanase [mailto:atan...@adobe.com] >>

Re: Spark/Kafka Streaming Job Gets Stuck

2015-10-29 Thread srungarapu vamsi
Other than @Adrian suggestions, check if the processing delay is more than the batch processing time. On Thu, Oct 29, 2015 at 2:23 AM, Adrian Tanase wrote: > Does it work as expected with smaller batch or smaller load? Could it be > that it's accumulating too many events over

Re: Spark/Kafka Streaming Job Gets Stuck

2015-10-28 Thread Adrian Tanase
Does it work as expected with smaller batch or smaller load? Could it be that it's accumulating too many events over 3 minutes? You could also try increasing the parallelism via repartition to ensure smaller tasks that can safely fit in working memory. Sent from my iPhone > On 28 Oct 2015, at

Re: Spark-Kafka Connector issue

2015-09-29 Thread Cody Koeninger
> > Sent from Outlook <http://taps.io/outlookmobile> > > _ > From: Cody Koeninger <c...@koeninger.org> > Sent: Tuesday, September 29, 2015 12:33 am > Subject: Re: Spark-Kafka Connector issue > To: Ratika Prasad <rpra...@couponsinc.

Re: spark kafka partitioning

2015-08-21 Thread Gaurav Agarwal
when i send the message from kafka topic having three partitions. Spark will listen the message when i say kafkautils.createStream or createDirectstSream have local[4] Now i want to see if spark will create partitions when it receive message from kafka using dstream, how and where ,prwhich method

Re: spark kafka partitioning

2015-08-20 Thread ayan guha
If you have 1 topic, that means you have 1 DStream, which will have a series of RDDs for each batch interval. In receiver-based integration, there is no direct relationship b/w Kafka paritions with spark partitions. in Direct approach, 1 partition will be created for each kafka partition. On Fri,

Re: spark kafka partitioning

2015-08-20 Thread Cody Koeninger
I'm not clear on your question, can you rephrase it? Also, are you talking about createStream or createDirectStream? On Thu, Aug 20, 2015 at 9:48 PM, Gaurav Agarwal gaurav130...@gmail.com wrote: Hello Regarding Spark Streaming and Kafka Partitioning When i send message on kafka topic with

Re: spark-kafka directAPI vs receivers based API

2015-08-10 Thread Cody Koeninger
For direct stream questions: https://github.com/koeninger/kafka-exactly-once Yes, it is used in production. For general spark streaming question: http://spark.apache.org/docs/latest/streaming-programming-guide.html On Mon, Aug 10, 2015 at 7:51 AM, Mohit Durgapal durgapalmo...@gmail.com

Re: Spark Kafka Direct Streaming

2015-07-07 Thread Tathagata Das
When you enable checkpointing by setting the checkpoint directory, you enable metadata checkpointing. Data checkpointing kicks in only if you are using a DStream operation that requires it, or you are enabling Write Ahead Logs to prevent data loss on driver failure. More discussion -

Re: Spark + Kafka

2015-04-01 Thread bit1...@163.com
Please make sure that you have given more cores than Receiver numbers. From: James King Date: 2015-04-01 15:21 To: user Subject: Spark + Kafka I have a simple setup/runtime of Kafka and Sprak. I have a command line consumer displaying arrivals to Kafka topic. So i know messages are being

Re: Spark + Kafka

2015-04-01 Thread James King
Thanks Saisai, Sure will do. But just a quick note that when i set master as local[*] Spark started showing Kafka messages as expected, so the problem in my view was to do with not enough threads to process the incoming data. Thanks. On Wed, Apr 1, 2015 at 10:53 AM, Saisai Shao

Re: Spark + Kafka

2015-04-01 Thread Saisai Shao
Would you please share your code snippet please, so we can identify is there anything wrong in your code. Beside would you please grep your driver's debug log to see if there's any debug log about Stream xxx received block xxx, this means that Spark Streaming is keeping receiving data from

RE: Spark + Kafka

2015-04-01 Thread Shao, Saisai
: James King [mailto:jakwebin...@gmail.com] Sent: Wednesday, April 1, 2015 6:59 PM To: Saisai Shao Cc: bit1...@163.com; user Subject: Re: Spark + Kafka This is the code. And I couldn't find anything like the log you suggested. public KafkaLogConsumer(int duration, String master

Re: Spark + Kafka

2015-04-01 Thread James King
Thank you bit1129, From looking at the web UI i can see 2 cores Also looking at http://spark.apache.org/docs/1.2.1/configuration.html But can't see obvious configuration for number of receivers can you help please. On Wed, Apr 1, 2015 at 9:39 AM, bit1...@163.com bit1...@163.com wrote:

Re: Spark + Kafka

2015-04-01 Thread James King
This is the code. And I couldn't find anything like the log you suggested. public KafkaLogConsumer(int duration, String master) { JavaStreamingContext spark = createSparkContext(duration, master); MapString, Integer topics = new HashMapString, Integer(); topics.put(test, 1);

Re: Spark + Kafka

2015-03-19 Thread James King
Thanks Khanderao. On Wed, Mar 18, 2015 at 7:18 PM, Khanderao Kand Gmail khanderao.k...@gmail.com wrote: I have used various version of spark (1.0, 1.2.1) without any issues . Though I have not significantly used kafka with 1.3.0 , a preliminary testing revealed no issues . - khanderao

Re: Spark + Kafka

2015-03-19 Thread James King
Many thanks all for the good responses, appreciated. On Thu, Mar 19, 2015 at 8:36 AM, James King jakwebin...@gmail.com wrote: Thanks Khanderao. On Wed, Mar 18, 2015 at 7:18 PM, Khanderao Kand Gmail khanderao.k...@gmail.com wrote: I have used various version of spark (1.0, 1.2.1) without

Re: Spark + Kafka

2015-03-18 Thread Jeffrey Jedele
Probably 1.3.0 - it has some improvements in the included Kafka receiver for streaming. https://spark.apache.org/releases/spark-release-1-3-0.html Regards, Jeff 2015-03-18 10:38 GMT+01:00 James King jakwebin...@gmail.com: Hi All, Which build of Spark is best when using Kafka? Regards jk

Re: Spark + Kafka

2015-03-18 Thread James King
Thanks Jeff, I'm planning to use it in standalone mode, OK will use hadoop 2.4 package. Chao! On Wed, Mar 18, 2015 at 10:56 AM, Jeffrey Jedele jeffrey.jed...@gmail.com wrote: What you call sub-category are packages pre-built to run on certain Hadoop environments. It really depends on where

Re: Spark + Kafka

2015-03-18 Thread Jeffrey Jedele
What you call sub-category are packages pre-built to run on certain Hadoop environments. It really depends on where you want to run Spark. As far as I know, this is mainly about the included HDFS binding - so if you just want to play around with Spark, any of the packages should be fine. I

Re: Spark + Kafka

2015-03-18 Thread Khanderao Kand Gmail
I have used various version of spark (1.0, 1.2.1) without any issues . Though I have not significantly used kafka with 1.3.0 , a preliminary testing revealed no issues . - khanderao On Mar 18, 2015, at 2:38 AM, James King jakwebin...@gmail.com wrote: Hi All, Which build of Spark is

Re: spark kafka batch integration

2014-12-15 Thread Cody Koeninger
For an alternative take on a similar idea, see https://github.com/koeninger/spark-1/tree/kafkaRdd/external/kafka/src/main/scala/org/apache/spark/rdd/kafka An advantage of the approach I'm taking is that the lower and upper offsets of the RDD are known in advance, so it's deterministic. I

Re: spark kafka batch integration

2014-12-15 Thread Koert Kuipers
thanks! i will take a look at your code. didn't realize there was already something out there. good point about upper offsets, i will add that feature to our version as well if you dont mind. i was thinking about making it deterministic for task failure transparently (even if no upper offsets

Re: Spark / Kafka connector - CDH5 distribution

2014-10-07 Thread Abraham Jacob
Thanks Sean, Sorry in my earlier question I meant to type CDH5.1.3 not CDH5.1.2 I presume it's included in spark-streaming_2.10-1.0.0-cdh5.1.3 But for some reason eclipse complains that import org.apache.spark.streaming.kafka cannon be resolved, even though I have included the

Re: Spark / Kafka connector - CDH5 distribution

2014-10-07 Thread Abraham Jacob
Never mind... my bad... made a typo. looks good. Thanks, On Tue, Oct 7, 2014 at 3:57 PM, Abraham Jacob abe.jac...@gmail.com wrote: Thanks Sean, Sorry in my earlier question I meant to type CDH5.1.3 not CDH5.1.2 I presume it's included in spark-streaming_2.10-1.0.0-cdh5.1.3 But for some

Re: Spark Kafka streaming - ClassNotFoundException: org.apache.spark.streaming.kafka.KafkaReceiver

2014-06-11 Thread gaurav.dasgupta
Thanks Tobias for replying. The problem was that, I have to provide the dependency jars' paths to the StreamingContext within the code. So, providing all the jar paths, resolved my problem. Refer the below code snippet: *JavaStreamingContext ssc = new JavaStreamingContext(args[0],

Re: Spark Kafka streaming - ClassNotFoundException: org.apache.spark.streaming.kafka.KafkaReceiver

2014-06-08 Thread Tobias Pfeiffer
Gaurav, I am not sure that the * expands to what you expect it to do. Normally the bash expands * to a space-separated string, not colon-separated. Try specifying all the jars manually, maybe? Tobias On Thu, Jun 5, 2014 at 6:45 PM, Gaurav Dasgupta gaurav.d...@gmail.com wrote: Hi, I have