Re: Relation between fetch.max.bytes, max.partition.fetch.bytes & max.poll.records

2023-12-09 Thread Debraj Manna
Can someone please clarify my below doubt? The same has been asked on stack overflow also. https://stackoverflow.com/q/77630586/785523 On Fri, 8 Dec, 2023, 21:33 Debraj Manna, wrote: > Thanks again. > > Another follow-up question, since max.poll.records has nothing to do with > fe

Re: Relation between fetch.max.bytes, max.partition.fetch.bytes & max.poll.records

2023-12-08 Thread Debraj Manna
1 PM Haruki Okada wrote: > poll-idle-ratio-avg=1.0 doesn't immediately mean fetch throughput problem > since if processing is very fast, the metric will always be near 1.0. > > 2023年12月4日(月) 13:09 Debraj Manna : > > > Thanks for the reply. > > > > I read KIP > >

Re: Relation between fetch.max.bytes, max.partition.fetch.bytes & max.poll.records

2023-12-03 Thread Debraj Manna
f all partition leaders are in the same broker, 40MB (2MB * 20 partition) > will be returned for a single fetch request. > > 2023年11月30日(木) 17:10 Debraj Manna : > > > The doc states that fetch.max.bytes & max.partition.fetch.bytes > > > > are not absolute max

Relation between fetch.max.bytes, max.partition.fetch.bytes & max.poll.records

2023-11-30 Thread Debraj Manna
The doc states that fetch.max.bytes & max.partition.fetch.bytes are not absolute maximum. If the first record batch in the first non-empty > partition of the fetch is larger than this limit, the batch will still be > returned to ensure that the consumer can make progress. I am getting a bit

How does Kafka Consumer send JoinRequest?

2023-11-26 Thread Debraj Manna
Can someone let me know if the JoinRequest is sent by the consumer from the polling/user thread or from the background heart-beat thread? If JoinRequest is being sent from the polling/user thread then in this case if the poll user thread takes more than max.poll.interval.secs then the consumer

Re: How does kafka consumer behave when consumer poll timeout has expired?

2023-11-03 Thread Debraj Manna
Debraj Manna, wrote: > Hi > > Can someone let me know how a consumer is expected to behave after the > below log? Will the consumer be considered dead and a new instance will be > spawned due to consumer group rebalancing? How is this behaviour with > RangeAssignor and Coopera

Re: Kafka Streams reaching ERROR state during rolling upgrade / restart of brokers

2023-11-02 Thread Debraj Manna
ion-handler via > > KafkaStreamssetUncaughtExceptionHandler(...) > > > -Matthias > > On 10/2/23 12:11 PM, Debraj Manna wrote: > > Are you suggesting to check the Kafka broker logs? I do not see any other > > errors logs on the client / application side. > > > > On Fri, 29 Sep, 2023, 22:01 Matthi

How does kafka consumer behave when consumer poll timeout has expired?

2023-11-02 Thread Debraj Manna
Hi Can someone let me know how a consumer is expected to behave after the below log? Will the consumer be considered dead and a new instance will be spawned due to consumer group rebalancing? How is this behaviour with RangeAssignor and CooperativeStickyAssginer? consumer poll timeout has

Re: Kafka Streams reaching ERROR state during rolling upgrade / restart of brokers

2023-10-02 Thread Debraj Manna
ROR state > to begin with? Maybe you need to increase/change some timeouts/retries > configs. > > The stack trace you shared, is a symptom, but not the root cause. > > -Matthias > > On 9/21/23 12:56 AM, Debraj Manna wrote: > > I am using Kafka broker 2.8.1 (fro

Kafka Streams reaching ERROR state during rolling upgrade / restart of brokers

2023-09-21 Thread Debraj Manna
I am using Kafka broker 2.8.1 (from AWS MSK) with Kafka clients and Kafka stream 3.5.1. I am observing that whenever some rolling upgrade is done on AWS MSK our stream application reaches an error state. I get the below exception on trying to query the state store caused by:

Reset Kafka TestContainers after each junit test without destroying container

2022-05-29 Thread Debraj Manna
Cross-posting from stackoverflow I am using kafka testcontainers with JUnit5. Can someone let me know how can I

Re: Clients may fetch incomplete set of topic partitions during cluster startup

2020-05-30 Thread Debraj Manna
gt; > If you're affected by the issue, it would be helpful to leave > a comment on the ticket to that effect. > > Thanks, > -John > > On Fri, May 29, 2020, at 00:05, Debraj Manna wrote: > > Anyone any update on my below query? > > > > On Thu, 2

Re: Clients may fetch incomplete set of topic partitions during cluster startup

2020-05-28 Thread Debraj Manna
Anyone any update on my below query? On Thu, 28 May 2020, 15:45 Debraj Manna, wrote: > Hi > > Is the below issue fixed in latest Kafka 2.5? > > https://issues.apache.org/jira/browse/KAFKA-8480 > > I am seeing this issue still open. So just confirming before upgrading

Clients may fetch incomplete set of topic partitions during cluster startup

2020-05-28 Thread Debraj Manna
Hi Is the below issue fixed in latest Kafka 2.5? https://issues.apache.org/jira/browse/KAFKA-8480 I am seeing this issue still open. So just confirming before upgrading Kafka to the latest. Thanks,

Re: Sort data across partitions and put it in another topic

2020-01-16 Thread Debraj Manna
eir destination partition. > > > On Thu, Jan 16, 2020 at 10:20 AM Debraj Manna > wrote: > > > Just to add when this operation will be going on no new data will be > added > > to original Kafka topic. I am trying to avoid buffering all data to a > > temporary datastore

Re: Sort data across partitions and put it in another topic

2020-01-16 Thread Debraj Manna
Just to add when this operation will be going on no new data will be added to original Kafka topic. I am trying to avoid buffering all data to a temporary datastore to sort. On Thu, 16 Jan 2020, 23:14 Debraj Manna, wrote: > Hi > > I have a Kafka topic with X partitions. Each me

Sort data across partitions and put it in another topic

2020-01-16 Thread Debraj Manna
Hi I have a Kafka topic with X partitions. Each message has a timestamp, ts. Can someone suggest me some way of sorting all the messages (based on ts) across all partitions and putting it in a new topic with Y partitions (Y < X ) using Kafka java client? Thanks

Re: Running Kafka Stream Application in YARN

2019-11-09 Thread Debraj Manna
Anyone any update on this? On Fri, 8 Nov 2019, 15:56 Debraj Manna, wrote: > Hi > > Is there any documentation or link I can refer to for the steps for > deploying the Kafka Streams application in YARN? > > Kafka Client - 0.11.0.3 > Kafka Broker - 2.2.1 > YARN - 2.6.0 >

Running Kafka Stream Application in YARN

2019-11-08 Thread Debraj Manna
Hi Is there any documentation or link I can refer to for the steps for deploying the Kafka Streams application in YARN? Kafka Client - 0.11.0.3 Kafka Broker - 2.2.1 YARN - 2.6.0

Re: Kafka 2.2.1 with OpenJDK 11

2019-10-26 Thread Debraj Manna
Thanks Jonathan. On Sun 27 Oct, 2019, 12:13 AM Jonathan Santilli, wrote: > Hello, > > I can confirm that the version 2.2.0 supports OpenJDK 8 in the broker side, > we are using it on production. > > > Cheers! > -- > Jonathan > > On Sat, Oct 26, 2019, 4:47 PM D

Re: Kafka 2.2.1 with OpenJDK 11

2019-10-26 Thread Debraj Manna
Kafka latest doc mention about JDK 8. But make no mention of openjdk or Oracle JDK. Can someone confirm if JDK 8 means openjdk 8 is also supported? https://kafka.apache.org/documentation/#java On Fri 25 Oct, 2019, 9:17 PM Debraj Manna, wrote: > Thanks for the pointers. > > Does ka

Re: Kafka 2.2.1 with OpenJDK 11

2019-10-25 Thread Debraj Manna
1 with jdk11, but there’s definitely > a trunk build passing for jdl11 yesterday morning. > > > > On Fri, 25 Oct 2019 at 04:16, Debraj Manna > wrote: > > > Can you point me to the link where I have to check? > > > > On Thu 24 Oct, 2019, 7:54 PM M. Manna, wrote: &

Re: Kafka 2.2.1 with OpenJDK 11

2019-10-24 Thread Debraj Manna
Can you point me to the link where I have to check? On Thu 24 Oct, 2019, 7:54 PM M. Manna, wrote: > Have you checked the Kafka build 2.3.1 RC2 which everyone is currently > voting for ? It’s worth checking for your question... > > Regards. > On Thu, 24 Oct 2019 at 13:31, Debraj

Kafka 2.2.1 with OpenJDK 11

2019-10-24 Thread Debraj Manna
Hi Does Kafka work with OpenJDK 11? I have seen the below issue which is resolved in 2.1. https://issues.apache.org/jira/browse/KAFKA-7264 But it does not mention about OpenJDK. Can someone confirm if Kafka 2.2.1 is supported with OpenJDK 11 also?

Re: KAFKA-7093 - Warn Messages in Kafka 1.1.0

2018-09-14 Thread Debraj Manna
Anyone on any thoughts on this? On Mon, Sep 3, 2018 at 11:28 PM Debraj Manna wrote: > Hi > > I am also observing lot of logs as discussed in > <https://issues.apache.org/jira/browse/KAFKA-7093>KAFKA-7093 > <https://issues.apache.org/jira/browse/KAFKA-7093> . Anyo

KAFKA-7093 - Warn Messages in Kafka 1.1.0

2018-09-03 Thread Debraj Manna
Hi I am also observing lot of logs as discussed in KAFKA-7093 . Anyone any thoughs? What does this denote? What does it effect and how to recover from this? Thanks,

Re: Issue with Samza 0.14.1 and Kafka 1.1.0 in handling OffSetOutOfRangeException

2018-08-26 Thread Debraj Manna
This is answered in :- https://stackoverflow.com/questions/51991805/samza-0-14-1-not-correctly-handling-offsetoutofrangeexception-exception/52028830#52028830 On Fri, Aug 24, 2018 at 9:55 AM Debraj Manna wrote: > Hi > > We are facing an issue with Samza 0.14.1 and Kafka 1.1.0. The det

Is Kafka broker 1.1.0 backward compatible with Java Kafka Client 0.10.1.1?

2018-08-24 Thread Debraj Manna
Hi Is Kafka broker 1.1.0 backward compatible with Java Kafka Client ? It is yes as per my understanding after reading this page but I want to confirm. Thanks,

Issue with Samza 0.14.1 and Kafka 1.1.0 in handling OffSetOutOfRangeException

2018-08-23 Thread Debraj Manna
Hi We are facing an issue with Samza 0.14.1 and Kafka 1.1.0. The details have been posted in samza mailing list and stackoverflow

Re: Partitions reassignment is failing in Kafka 1.1.0

2018-07-01 Thread Debraj Manna
uch case. > > On Sat, Jun 30, 2018 at 12:06 AM, Manikumar > wrote: > > > It will be taken as "any" directory for each replica, which means replica > > will placed on any one of the > > configured directory on that broker. > > > > Since it

Specifying negative number as partition in ProducerRecord prior to 1.1.0

2018-07-01 Thread Debraj Manna
In Kafka Client 1.1.0 a check is placed to not allow negative as partition in ProducerRecord. https://github.com/apache/kafka/blob/1.1/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java#L73 But I don;t see any such check in ProducerRecord in kafka client 0.10.0. Can

Re: Partitions reassignment is failing in Kafka 1.1.0

2018-06-30 Thread Debraj Manna
e from the json. > > On Sat, Jun 30, 2018 at 12:02 PM Debraj Manna > wrote: > > > It is problem on my side. The code was changing the replicas count but > not > > the log_dirs. Since I am migrating from 0.10 this part of the code was > not > > changed. > &g

Re: Partitions reassignment is failing in Kafka 1.1.0

2018-06-30 Thread Debraj Manna
:15 AM, Debraj Manna wrote: > I am generating the reassignent.json like below > > /home/ubuntu/deploy/kafka/bin/kafka-reassign-partitions.sh --zookeeper > 127.0.0.1:2181 --generate --topics-to-move-json-file > /home/ubuntu/deploy/kafka/topics_to_move.json --broker-list '%s' |tai

Re: Partitions reassignment is failing in Kafka 1.1.0

2018-06-29 Thread Debraj Manna
currently required that the replica has not already been > created on that broker. The replica will then be created in the specified > log directory on the broker later. So it appears reassignment json that is generated by kafka-reassign-partions.sh is creating an issue with logdirs. Is this

Re: Partitions reassignment is failing in Kafka 1.1.0

2018-06-29 Thread Debraj Manna
zookeeper.connection.timeout.ms=6000 unclean.leader.election.enable=false delete.topic.enable=false offsets.topic.replication.factor=1 transaction.state.log.replication.factor=1 transaction.state.log.min.isr=1 I have placed server.log from a broker at https://gist.github.com/debraj-manna

Re: Partitions reassignment is failing in Kafka 1.1.0

2018-06-29 Thread Debraj Manna
Hi Anyone any thoughts? On Fri 29 Jun, 2018, 11:07 PM Debraj Manna, wrote: > Hi > > I altered a topic like below in kafka 1.1.0 > > /home/ubuntu/deploy/kafka/bin/kafka-topics.sh --zookeeper 127.0.0.1:2181 > --alter --topic Topic3 --config min.insync.replicas=2 > >

Partitions reassignment is failing in Kafka 1.1.0

2018-06-29 Thread Debraj Manna
Hi I altered a topic like below in kafka 1.1.0 /home/ubuntu/deploy/kafka/bin/kafka-topics.sh --zookeeper 127.0.0.1:2181 --alter --topic Topic3 --config min.insync.replicas=2 But whenever I am trying to verify the reassignment it is showing the below exception

Is 0.11 inter broker protocol and log message format backward compatible with 0.10.0 ?

2018-06-18 Thread Debraj Manna
Hi I have seen the ugrade doc for 0.11. Let's say I am having a 3 node kafka broker cluster running 0.10 and no inter.broker.protocol.version and log.message.format.version not added to the server.properties in any of the brokers. Can

Re: new pom.xml for samza-parent-project

2018-06-17 Thread Debraj Manna
Thanks Martin for looking into this. On Sun, Jun 17, 2018 at 4:33 PM, Martin Gainty wrote: > you're welcome > > > bye > __ > > > > ------ > *From:* Debraj Manna > *Sent:* Sunday, June 17, 2

Re: NoClassDefFoundError TopicExistxException Samza 0.14.2 and Kafka 0.10.0.0

2018-06-15 Thread Debraj Manna
Please find the kafka and samza dependencies in the below link https://gist.github.com/debraj-manna/d51af1896d74b68da55f0ef252886692 On Fri, Jun 15, 2018 at 9:14 PM, Martin Gainty wrote: > unable to determine execution scenario without seeing pom.xml > > > please supply pom.xml

Re: NoClassDefFoundError TopicExistxException Samza 0.14.2 and Kafka 0.10.0.0

2018-06-15 Thread Debraj Manna
, 2018 at 8:31 PM, Martin Gainty wrote: > a strong possibility is that kafka-core is missing as a dependency for > samza > > please supply samza pom.xml > > > Martin > __ > > > > > ________ >

NoClassDefFoundError TopicExistxException Samza 0.14.2 and Kafka 0.10.0.0

2018-06-15 Thread Debraj Manna
Hi I have posted the same in d...@samza.apache.org , posting it here also if anyone is aware. *I am trying to use samza 0.14.2 with Kafka Client 0.10.0 . I am getting the below exception. Is this related to https://issues.apache.org/jira/browse/SAMZA-1509

Re: Kafka bootstrap broker disconnected

2018-03-23 Thread Debraj Manna
om > your zookeeper node. Check if your zookeeper is alive, if you kafka node > can reach zookeeper and vice-versa. > > But your zookeeper's ip <+911727124597> is very strange ... > > > best regards, > > > Adrien > > > De

Re: Kafka bootstrap broker disconnected

2018-03-23 Thread Debraj Manna
=localhost:2181 zookeeper.connection.timeout.ms=6000 num.network.threads=20 log.roll.hours=24 log.segment.bytes=10737418240 listeners=PLAINTEXT://172.71.245.97:9092 advertised.listeners=PLAINTEXT://172.71.245.97:9092 On Fri, Mar 23, 2018 at 11:01 AM, Debraj Manna <subharaj.ma...@gmail.com> wrote:

Kafka bootstrap broker disconnected

2018-03-22 Thread Debraj Manna
Hi In my java Kafka client I am seeing the below logs and I am not seeing any data being written to Kafka 0.10. Can someone let me know what is going wrong? kafka “WARN [2018-03-22 20 <+912018032220>:31:27]

Kafka broker throwing ConfigException Invalid value configuration log.segment.bytes: Not a number of type INT

2018-02-19 Thread Debraj Manna
Cross-posting from stackoverflow I have a single node kafka broker and single node zookeeper with the server.properties like below broker.id=0 num.network.threads=3

Re: Java Consumer Not reading message -

2018-02-18 Thread Debraj Manna
lso inspect your consumer's position via KafkaConsumer#position() to see where the consumer actually is in the topic. -Matthias On 2/16/18 5:13 AM, Debraj Manna wrote: > I have posted the same question in stackoverflow also. But I have not got > any reply there also > > https://stackoverflow.

Re: Java Consumer Not reading message -

2018-02-16 Thread Debraj Manna
I have posted the same question in stackoverflow also. But I have not got any reply there also https://stackoverflow.com/questions/48826279/kafka-0-10-java-consumer-not-reading-message-from-topic On Fri, Feb 16, 2018 at 5:23 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > I have

Java Consumer Not reading message -

2018-02-16 Thread Debraj Manna
I have a simple java producer like below public class Producer { private final static String TOPIC = "my-example-topi8"; private final static String BOOTSTRAP_SERVERS = "localhost:8092"; public static void main( String[] args ) throws Exception { Producer producer

Re: Running Kafka 1.0 binaries with inter.broker.protocol.version = 0.10

2018-01-27 Thread Debraj Manna
Anyone any thoughts? On Fri, Dec 8, 2017 at 6:53 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > Can anyone let me know if I set both inter.broker.protocol.version & > log.message.format.version to 0.10 with the updated 1.0 binaries ? How > are the Kafka brokers

Consumer behavior when Kafka rolls the old log file

2018-01-02 Thread Debraj Manna
Can someone let me know how does consumer behaves when Kafka rolls the old log file (e.g. i.e. INFO Rolled new log segment for 'topic-{partition}') while the consumer is still consuming the log segment ?

Error while fetching metadata with correlation id 3

2017-12-28 Thread Debraj Manna
Hi I am seeing an warning like below and my kafka java producer client is not able to write to kafka broker. (Kafka version 0.10.0 both client & server) WARN Error while fetching metadata with correlation id 3 : {abcdef=LEADER_NOT_AVAILABLE} - OS - 14.04.1-Ubuntu - Java - 8 In kafka

Re: kafka-client throwing IllegalStateException on calling wait

2017-12-28 Thread Debraj Manna
on-how-to-solve-illegalmonitorstateexception/ > ? > > You didn't include the whole code w.r.t. shadowKafkaProducer > If you need more help, please consider including more of your code. > > Cheers > > On Wed, Dec 27, 2017 at 5:32 AM, Debraj Manna <subharaj.ma...@gmail.com> > wrote: &

Re: kafka-client throwing IllegalStateException on calling wait

2017-12-27 Thread Debraj Manna
Anyone any thoughts? On Wed, Dec 27, 2017 at 7:02 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > Cross-posting from stackoverflow > <https://stackoverflow.com/questions/47992916/kafka-client-throwing-illegalstateexception-on-calling-wait> > > Kafka Client 0.10.0

kafka-client throwing IllegalStateException on calling wait

2017-12-27 Thread Debraj Manna
Cross-posting from stackoverflow Kafka Client 0.10.0.0 is throwing the below IllegalStateException whenever I am calling wait() ERROR [2017-12-27 09:55:48] c.v.g.u.UploadHandler:[?:?:?] -

Re: Running Kafka 1.0 binaries with inter.broker.protocol.version = 0.10

2017-12-08 Thread Debraj Manna
Can anyone let me know if I set both inter.broker.protocol.version & log.message.format.version to 0.10 with the updated 1.0 binaries ? How are the Kafka brokers supposed to behave? On Thu, Dec 7, 2017 at 5:10 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > Hi > > Anyone

Re: Running Kafka 1.0 binaries with inter.broker.protocol.version = 0.10

2017-12-07 Thread Debraj Manna
Hi Anyone any thoughts on my last query? On Wed, Dec 6, 2017 at 11:09 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > Thanks Manikumar for replying. One more query regarding your first reply > > What if I set both inter.broker.protocol.version & log.message.forma

Re: Running Kafka 1.0 binaries with inter.broker.protocol.version = 0.10

2017-12-06 Thread Debraj Manna
s given in the docs. > > On Wed, Dec 6, 2017 at 11:21 AM, Debraj Manna <subharaj.ma...@gmail.com> > wrote: > > > Hi > > > > Anyone any thoughts? > > > > > > > > On Tue, Dec 5, 2017 at 8:38 PM, Debraj Manna <subharaj.ma...@gmail.c

Re: Running Kafka 1.0 binaries with inter.broker.protocol.version = 0.10

2017-12-05 Thread Debraj Manna
Hi Anyone any thoughts? On Tue, Dec 5, 2017 at 8:38 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > Hi > > Regarding the Kafka Rolling Upgrade steps as mentioned in the doc > <https://kafka.apache.org/documentation/#upgrade> > > Can you let me know h

Running Kafka 1.0 binaries with inter.broker.protocol.version = 0.10

2017-12-05 Thread Debraj Manna
Hi Regarding the Kafka Rolling Upgrade steps as mentioned in the doc Can you let me know how is Kafka supposed to behave if the binaries are upgraded to the latest 1.0 but inter.broker.protocol.version still points to 0.10 in all the brokers?

Kafka Topic Exists Exception - On upgrading to 1.0

2017-11-29 Thread Debraj Manna
Hi I am trying to upgrade a single node kafka broker to latest 1.0 from 0.10. The steps followed 1. Stopped Kafka Broker 2. Replaced bin/, libs & site-docs/ with the latest 3. Started Kafka But I am seeing the below exception in logs - Anyone any thoughts? How can I get around this?

Re: Recommended settings for Internal Topics

2017-11-27 Thread Debraj Manna
h error. > > Jakub > > On Sat, Nov 25, 2017 at 6:44 AM, Debraj Manna <subharaj.ma...@gmail.com> > wrote: > > > Anyone any thoughts? > > > > If I am not changing this value after upgrade to 1.0 in a single node > kafka > > broker. It will take the defa

Re: Recommended settings for Internal Topics

2017-11-24 Thread Debraj Manna
Anyone any thoughts? If I am not changing this value after upgrade to 1.0 in a single node kafka broker. It will take the default value of 3 . So what will be the behavior in this case? On Fri, Nov 24, 2017 at 3:57 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > Hi > > I am

Recommended settings for Internal Topics

2017-11-24 Thread Debraj Manna
Hi I am migrating from Kafka 0.10 to the latest 1.0 . I did not set any value for these fields in Kafka 0.10. Can some one let me know what is the recommended settings for a 3 node broker cluster & for a single node broker cluster for the below internal topic settings - -

Re: Running apache samza with Kafka Client 1.0 - JIRA - SAMZA - 1418

2017-11-22 Thread Debraj Manna
data is the public version. Either way, all > these classes are used by the deprecated Scala consumers and will be > removed in a future version. It would be great if Samza migrated to the > Java consumer. > > Ismael > > On Wed, Nov 22, 2017 at 2:52 PM, Debraj Manna <s

Fwd: Running apache samza with Kafka Client 1.0 - JIRA - SAMZA - 1418

2017-11-22 Thread Debraj Manna
Hi I posted the same query in samza mailing list. But I did not get any reply. Anyone has any thoughts? Sent from GMail on Android -- Forwarded message -- From: "Debraj Manna" <subharaj.ma...@gmail.com> Date: Nov 21, 2017 5:34 PM Subject: Running apache samza w

Logstash 1.5.3 with Kafka 0.9’s quota

2015-12-20 Thread Debraj Manna
In Kafka 0.9 support for quota is added as mentioned here . I am having some doubts about the behavior of quota in accordance with logstash. So in Kafka 0.9 quota can be configured for each client-id. If one of the client violates the

Maximum Topic Length in Kafka

2015-11-28 Thread Debraj Manna
Hi, Can some one please let me know the following:- 1. Is it possible to specify maximum length of a particular topic ( in terms of number of messages ) in kafka ? 2. Also how does Kafka behave when a particular topic gets full? 3. Can the producer be blocked if a topic get full

Re: Maximum Topic Length in Kafka

2015-11-28 Thread Debraj Manna
Management Solr & Elasticsearch Support Sematext <http://sematext.com/> | Contact <http://sematext.com/about/contact.html> On Sat, Nov 28, 2015 at 2:13 PM, Debraj Manna <subharaj.ma...@gmail.com> wrote: > Hi, > > Can some one please let me know the followi

Metrics to monitor in Kafka

2015-08-25 Thread Debraj Manna
Hi, What are the important metrics to monitor in Kafka (as explained here https://kafka.apache.org/documentation.html#monitoring) and raise alarms when the value reaches certain value? I am looking for some guidelines similar to this excellent monitoring doc