[VOTE] KIP-596 Safely abort Producer transactions during application shutdown

2020-05-15 Thread
Hi all,

I'd like to start a vote for KIP-586. You can find the link for this KIP
here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-596%3A+Safely+abort+Producer+transactions+during+application+shutdown

Thanks.


Re: [DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-05-05 Thread
Thanks for the comment Matthias.

In fact, I cannot think of why we cannot close the producer no matter what.
On the other hand, it is also okay to reuse the producer when the error is
not fatal. @Guozhang Wang   @Boyang Chen


Matthias J. Sax  于2020年5月1日周五 上午7:52写道:

> Thanks for the KIP. Make sense to me. I think you can start a vote.
>
> One minor comment about the code example: From my understanding, a
> producer should always be closed (independent if there was no error, a
> transient error, or a fatal error). If that is correct, than the code
> example seems to be miss-leading?
>
>
> -Matthias
>
> On 4/25/20 6:08 PM, 张祥 wrote:
> > Sorry, but this KIP is still open to discussion, any comments and ideas
> > would be appreciated, Thanks.
> >
> > 张祥  于2020年4月17日周五 下午1:04写道:
> >
> >> Guozhang, thanks for the valuable suggestion.
> >>
> >> A new part called "suggested coding pattern" has been added and I copy
> the
> >> core code here:
> >>
> >> try {
> >> producer.beginTransaction();
> >> for (int i = 0; i < 100; i++)
> >> producer.send(new ProducerRecord<>("my-topic",
> >> Integer.toString(i), Integer.toString(i)));
> >> producer.commitTransaction();
> >> } catch (Exception e) {
> >> producer.abortTransaction();
> >> if(e instanceof IllegalStateException ||
> >> e instanceof ProducerFencedException ||
> >> e instanceof UnsupportedVersionException ||
> >> e instanceof AuthorizationException ||
> >> e instanceof OutOfOrderSequenceException) {
> >> producer.close();
> >> }
> >> }
> >>
> >> As you can see, in the catch block,  all fatal exceptions need to be
> >> listed, I am not sure I have listed all of them and I wonder if there
> is a
> >> better way to do this.
> >>
> >>
> >> Guozhang Wang  于2020年4月17日周五 上午8:50写道:
> >>
> >>> Xiang, thanks for the written KIP. I just have one meta comment and
> >>> otherwise it looks good to me: could you also add a section about
> >>> suggested
> >>> coding patterns (especially how try - catch should be implemented) as
> we
> >>> discussed on the JIRA to the wiki page as well?
> >>>
> >>> And please also note that besides the javadoc of the function, on top
> of
> >>> the KafkaProducer class there are also comments regarding example
> snippet:
> >>>
> >>> ```
> >>>
> >>> * 
> >>> * {@code
> >>> * Properties props = new Properties();
> >>> * props.put("bootstrap.servers", "localhost:9092");
> >>> * props.put("transactional.id", "my-transactional-id");
> >>> * Producer producer = new KafkaProducer<>(props, new
> >>> StringSerializer(), new StringSerializer());
> >>> *
> >>> * producer.initTransactions();
> >>> *
> >>> * try {
> >>> * producer.beginTransaction();
> >>> * for (int i = 0; i < 100; i++)
> >>> * producer.send(new ProducerRecord<>("my-topic",
> >>> Integer.toString(i), Integer.toString(i)));
> >>> * producer.commitTransaction();
> >>> * } catch (ProducerFencedException | OutOfOrderSequenceException |
> >>> AuthorizationException e) {
> >>> * // We can't recover from these exceptions, so our only option is
> >>> to close the producer and exit.
> >>> * producer.close();
> >>> * } catch (KafkaException e) {
> >>> * // For all other exceptions, just abort the transaction and try
> >>> again.
> >>> * producer.abortTransaction();
> >>> * }
> >>> * producer.close();
> >>>
> >>> * } 
> >>> ```
> >>>
> >>> I think with this change we do not need to educate users that they
> should
> >>> distinguish the types of exceptions when calling `abortTxn`, instead
> they
> >>> only need to depend on the exception to decide whether to `close` the
> >>> producer, so the above recommendation could look like:
> >>>
> >>> try {
> >>>
> >>> } catch {Exception e} {
> >>>
> >>> producer.abortTxn;
> >>>
> >>> if (e instanceof /*fatal exceptions*/) {
> >>> producer.close();
> 

Re: [DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-04-25 Thread
Sorry, but this KIP is still open to discussion, any comments and ideas
would be appreciated, Thanks.

张祥  于2020年4月17日周五 下午1:04写道:

> Guozhang, thanks for the valuable suggestion.
>
> A new part called "suggested coding pattern" has been added and I copy the
> core code here:
>
> try {
> producer.beginTransaction();
> for (int i = 0; i < 100; i++)
> producer.send(new ProducerRecord<>("my-topic",
> Integer.toString(i), Integer.toString(i)));
> producer.commitTransaction();
> } catch (Exception e) {
> producer.abortTransaction();
> if(e instanceof IllegalStateException ||
> e instanceof ProducerFencedException ||
> e instanceof UnsupportedVersionException ||
> e instanceof AuthorizationException ||
> e instanceof OutOfOrderSequenceException) {
> producer.close();
> }
> }
>
> As you can see, in the catch block,  all fatal exceptions need to be
> listed, I am not sure I have listed all of them and I wonder if there is a
> better way to do this.
>
>
> Guozhang Wang  于2020年4月17日周五 上午8:50写道:
>
>> Xiang, thanks for the written KIP. I just have one meta comment and
>> otherwise it looks good to me: could you also add a section about
>> suggested
>> coding patterns (especially how try - catch should be implemented) as we
>> discussed on the JIRA to the wiki page as well?
>>
>> And please also note that besides the javadoc of the function, on top of
>> the KafkaProducer class there are also comments regarding example snippet:
>>
>> ```
>>
>> * 
>> * {@code
>> * Properties props = new Properties();
>> * props.put("bootstrap.servers", "localhost:9092");
>> * props.put("transactional.id", "my-transactional-id");
>> * Producer producer = new KafkaProducer<>(props, new
>> StringSerializer(), new StringSerializer());
>> *
>> * producer.initTransactions();
>> *
>> * try {
>> * producer.beginTransaction();
>> * for (int i = 0; i < 100; i++)
>> * producer.send(new ProducerRecord<>("my-topic",
>> Integer.toString(i), Integer.toString(i)));
>> * producer.commitTransaction();
>> * } catch (ProducerFencedException | OutOfOrderSequenceException |
>> AuthorizationException e) {
>> * // We can't recover from these exceptions, so our only option is
>> to close the producer and exit.
>> * producer.close();
>> * } catch (KafkaException e) {
>> * // For all other exceptions, just abort the transaction and try
>> again.
>> * producer.abortTransaction();
>> * }
>> * producer.close();
>>
>> * } 
>> ```
>>
>> I think with this change we do not need to educate users that they should
>> distinguish the types of exceptions when calling `abortTxn`, instead they
>> only need to depend on the exception to decide whether to `close` the
>> producer, so the above recommendation could look like:
>>
>> try {
>>
>> } catch {Exception e} {
>>
>> producer.abortTxn;
>>
>> if (e instanceof /*fatal exceptions*/) {
>> producer.close();
>> }
>> }
>>
>>
>> Guozhang
>>
>> On Thu, Apr 16, 2020 at 12:14 AM 张祥  wrote:
>>
>> > Thanks for the structure change Boyang. And I agree with you on the weak
>> > proposal part, I have adjusted it according to your suggestion. Thanks
>> > again!
>> >
>> > Boyang Chen  于2020年4月16日周四 下午2:39写道:
>> >
>> > > Thanks for the KIP Xiang!
>> > >
>> > > I think the motivation looks good, and I just did a slight structure
>> > change
>> > > to separate "Proposed Changes" and "Public Interfaces", hope you don't
>> > > mind.
>> > >
>> > > However, "we can determine whether the producer client is already in
>> > error
>> > > state in abortTransaction" sounds a bit weak about the actual
>> proposal,
>> > > instead we could propose something as "we would remember whether a
>> fatal
>> > > exception has already been thrown to the application level, so that in
>> > > abort transaction we will not throw again, thus making the function
>> safe
>> > to
>> > > be called in an error state".
>> > >
>> > > Other than that, I think the KIP is in pretty good shape.
>> > >
>> > > Boyang
>> > >
>> > > On Wed, Apr 15, 2020 at 7:07 PM 张祥  wrote:
>> > >
>> > > > Hi everyone,
>> > > >
>> > > > I have opened a small KIP about safely aborting transaction during
>> > > > shutdown. I'd like to use this thread to discuss about it and any
>> > > feedback
>> > > > is appreciated (sorry for earlier KIP number mistake). Here is a
>> link
>> > to
>> > > > KIP-596 :
>> > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-596%3A+Safely+abort+Producer+transactions+during+application+shutdown
>> > > >
>> > > > Thank you!
>> > > >
>> > >
>> >
>>
>>
>> --
>> -- Guozhang
>>
>


Re: [DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-04-16 Thread
Guozhang, thanks for the valuable suggestion.

A new part called "suggested coding pattern" has been added and I copy the
core code here:

try {
producer.beginTransaction();
for (int i = 0; i < 100; i++)
producer.send(new ProducerRecord<>("my-topic", Integer.toString(i),
Integer.toString(i)));
producer.commitTransaction();
} catch (Exception e) {
producer.abortTransaction();
if(e instanceof IllegalStateException ||
e instanceof ProducerFencedException ||
e instanceof UnsupportedVersionException ||
e instanceof AuthorizationException ||
e instanceof OutOfOrderSequenceException) {
producer.close();
}
}

As you can see, in the catch block,  all fatal exceptions need to be
listed, I am not sure I have listed all of them and I wonder if there is a
better way to do this.


Guozhang Wang  于2020年4月17日周五 上午8:50写道:

> Xiang, thanks for the written KIP. I just have one meta comment and
> otherwise it looks good to me: could you also add a section about suggested
> coding patterns (especially how try - catch should be implemented) as we
> discussed on the JIRA to the wiki page as well?
>
> And please also note that besides the javadoc of the function, on top of
> the KafkaProducer class there are also comments regarding example snippet:
>
> ```
>
> * 
> * {@code
> * Properties props = new Properties();
> * props.put("bootstrap.servers", "localhost:9092");
> * props.put("transactional.id", "my-transactional-id");
> * Producer producer = new KafkaProducer<>(props, new
> StringSerializer(), new StringSerializer());
> *
> * producer.initTransactions();
> *
> * try {
> * producer.beginTransaction();
> * for (int i = 0; i < 100; i++)
> * producer.send(new ProducerRecord<>("my-topic",
> Integer.toString(i), Integer.toString(i)));
> * producer.commitTransaction();
> * } catch (ProducerFencedException | OutOfOrderSequenceException |
> AuthorizationException e) {
> * // We can't recover from these exceptions, so our only option is
> to close the producer and exit.
> * producer.close();
> * } catch (KafkaException e) {
> * // For all other exceptions, just abort the transaction and try
> again.
> * producer.abortTransaction();
> * }
> * producer.close();
>
> * } 
> ```
>
> I think with this change we do not need to educate users that they should
> distinguish the types of exceptions when calling `abortTxn`, instead they
> only need to depend on the exception to decide whether to `close` the
> producer, so the above recommendation could look like:
>
> try {
>
> } catch {Exception e} {
>
> producer.abortTxn;
>
> if (e instanceof /*fatal exceptions*/) {
> producer.close();
> }
> }
>
>
> Guozhang
>
> On Thu, Apr 16, 2020 at 12:14 AM 张祥  wrote:
>
> > Thanks for the structure change Boyang. And I agree with you on the weak
> > proposal part, I have adjusted it according to your suggestion. Thanks
> > again!
> >
> > Boyang Chen  于2020年4月16日周四 下午2:39写道:
> >
> > > Thanks for the KIP Xiang!
> > >
> > > I think the motivation looks good, and I just did a slight structure
> > change
> > > to separate "Proposed Changes" and "Public Interfaces", hope you don't
> > > mind.
> > >
> > > However, "we can determine whether the producer client is already in
> > error
> > > state in abortTransaction" sounds a bit weak about the actual proposal,
> > > instead we could propose something as "we would remember whether a
> fatal
> > > exception has already been thrown to the application level, so that in
> > > abort transaction we will not throw again, thus making the function
> safe
> > to
> > > be called in an error state".
> > >
> > > Other than that, I think the KIP is in pretty good shape.
> > >
> > > Boyang
> > >
> > > On Wed, Apr 15, 2020 at 7:07 PM 张祥  wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > I have opened a small KIP about safely aborting transaction during
> > > > shutdown. I'd like to use this thread to discuss about it and any
> > > feedback
> > > > is appreciated (sorry for earlier KIP number mistake). Here is a link
> > to
> > > > KIP-596 :
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-596%3A+Safely+abort+Producer+transactions+during+application+shutdown
> > > >
> > > > Thank you!
> > > >
> > >
> >
>
>
> --
> -- Guozhang
>


Restart without shutdown log

2020-04-16 Thread
Hi, I am using Ambari to manage Kafka, info as listed below:

Ambari version: 2.7.4.0
Kafka version: 2.0.0

The problem I ran into is that one broker restarts without shutdown log,
which makes it difficult to track down the reason. The related logs are
as follows,  in which I cannot find "shut down" message and it seems that
there is a log pause around 1.5 minutes before a sudden restart.

[2020-04-14 08:01:44,214] INFO [TransactionCoordinator id=1008] Initialized
transactionalId Source: Custom Source -> Sink:
test-d06beb747c9d00565739f0dfcdd14614-12 with producerId 217004 and
producer epoch 31222 on partition __transaction_state-41
(kafka.coordinator.transaction.TransactionCoordinator)
*[2020-04-14 08:01:45,276]* INFO [TransactionCoordinator id=1008]
Initialized transactionalId Source: Custom Source -> Filter -> Map ->
Filter -> Sink: Unnamed-9ab9edcfebd7d79af65d59b6e0b37d6f-23 with producerId
217030 and producer epoch 19220 on partition __transaction_state-1
(kafka.coordinator.transaction.TransactionCoordinator)
*[2020-04-14 08:03:23,026]* INFO Registered
kafka:type=kafka.Log4jController MBean
(kafka.utils.Log4jControllerRegistration$)
*[2020-04-14 08:03:23,678] INFO starting (kafka.server.KafkaServer)*
[2020-04-14 08:03:23,679] INFO Connecting to zookeeper on
luoge-kafka-03:2181,luoge-kafka-02:2181,luoge-kafka-04:2181,luoge-kafka-05:2181,luoge-kafka-06:2181
(kafka.server.KafkaServer)
[2020-04-14 08:03:23,774] INFO [ZooKeeperClient] Initializing a new session
to
luoge-kafka-03:2181,luoge-kafka-02:2181,luoge-kafka-04:2181,luoge-kafka-05:2181,luoge-kafka-06:2181.
(kafka.zookeeper.ZooKeeperClient)
[2020-04-14 08:03:23,819] INFO [ZooKeeperClient] Waiting until connected.
(kafka.zookeeper.ZooKeeperClient)
[2020-04-14 08:03:23,874] ERROR [ZooKeeperClient] Auth failed.
(kafka.zookeeper.ZooKeeperClient)
[2020-04-14 08:03:23,941] INFO [ZooKeeperClient] Connected.
(kafka.zookeeper.ZooKeeperClient)
[2020-04-14 08:03:24,382] INFO Cluster ID = J7raFmuqQ8mh_7DwZArH0A
(kafka.server.KafkaServer)
[2020-04-14 08:03:24,629] INFO KafkaConfig values:

Any insight would be appreciated, thanks.


endless authorizer log

2020-04-16 Thread
I am using Ambari to manage Kafka, info listed below:

Ambari version: 2.7.4.0
Kafka version: 2.0.0
broker number: 10

On every broker, authorizer logger keeps outputting following logs:

 [2020-04-14 07:56:40,214] INFO Principal = User:xxx is Denied Operation =
Describe from host = 10.90.1.213 on resource = Topic:LITERAL:some_topic
(kafka.authorizer.logger)
 [2020-04-14 07:56:40,214] INFO Principal = User:xxx is Denied Operation =
Describe from host = 10.90.1.213 on resource = Topic:LITERAL:some_topic
(kafka.authorizer.logger)
 [2020-04-14 07:56:40,214] INFO Principal = User: is Denied Operation =
Describe from host = 10.90.1.213 on resource = Topic:LITERAL:some_topic
(kafka.authorizer.logger)

I would like to know why and how to prevent this. Thanks.


Re: [DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-04-16 Thread
Thanks for the structure change Boyang. And I agree with you on the weak
proposal part, I have adjusted it according to your suggestion. Thanks
again!

Boyang Chen  于2020年4月16日周四 下午2:39写道:

> Thanks for the KIP Xiang!
>
> I think the motivation looks good, and I just did a slight structure change
> to separate "Proposed Changes" and "Public Interfaces", hope you don't
> mind.
>
> However, "we can determine whether the producer client is already in error
> state in abortTransaction" sounds a bit weak about the actual proposal,
> instead we could propose something as "we would remember whether a fatal
> exception has already been thrown to the application level, so that in
> abort transaction we will not throw again, thus making the function safe to
> be called in an error state".
>
> Other than that, I think the KIP is in pretty good shape.
>
> Boyang
>
> On Wed, Apr 15, 2020 at 7:07 PM 张祥  wrote:
>
> > Hi everyone,
> >
> > I have opened a small KIP about safely aborting transaction during
> > shutdown. I'd like to use this thread to discuss about it and any
> feedback
> > is appreciated (sorry for earlier KIP number mistake). Here is a link to
> > KIP-596 :
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-596%3A+Safely+abort+Producer+transactions+during+application+shutdown
> >
> > Thank you!
> >
>


[DISCUSS] KIP-596 Safely abort Producer transactions during application shutdown

2020-04-15 Thread
Hi everyone,

I have opened a small KIP about safely aborting transaction during
shutdown. I'd like to use this thread to discuss about it and any feedback
is appreciated (sorry for earlier KIP number mistake). Here is a link to
KIP-596 :
https://cwiki.apache.org/confluence/display/KAFKA/KIP-596%3A+Safely+abort+Producer+transactions+during+application+shutdown

Thank you!


Re: [DISCUSS] KIP-594 Safely abort Producer transactions during application shutdown

2020-04-15 Thread
Thanks for reminding, I will change my KIP number and start a new thread
now.

Kowshik Prakasam  于2020年4月16日周四 上午9:49写道:

> Hi,
>
> It appears "KIP-594" is already taken. Please see this existing link:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-594%3A+Expose+output+topic+names+from+TopologyTestDriver
>  .
> To avoid a duplicate, please change your KIP number to pick the next
> available KIP number, as mentioned here:
>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals#KafkaImprovementProposals-KIPround-up
>
> Once you are done, I'd suggest that you please start a separate discussion
> thread with the new KIP number.
>
>
> Cheers,
> Kowshik
>
>
> On Wed, Apr 15, 2020 at 6:42 PM 张祥  wrote:
>
> > Hi everyone,
> >
> > I have opened a small KIP about safely aborting transaction during
> > shutdown. I'd like to use this thread to discuss about it and any
> feedback
> > is appreciated. Here is a link to KIP-594 :
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-594%3A+Safely+abort+Producer+transactions+during+application+shutdown
> >
> > Thank you!
> >
>


[DISCUSS] KIP-594 Safely abort Producer transactions during application shutdown

2020-04-15 Thread
Hi everyone,

I have opened a small KIP about safely aborting transaction during
shutdown. I'd like to use this thread to discuss about it and any feedback
is appreciated. Here is a link to KIP-594 :
https://cwiki.apache.org/confluence/display/KAFKA/KIP-594%3A+Safely+abort+Producer+transactions+during+application+shutdown

Thank you!


Re: Permission to create a KIP

2020-04-13 Thread
Thanks Guozhang.

Guozhang Wang  于2020年4月14日周二 上午12:52写道:

> I've added your id to the apache wiki space. You should be able to create
> new pages now.
>
> On Sun, Apr 12, 2020 at 10:55 PM 张祥  wrote:
>
> >  I just registered a new account with xiangzhang1...@gmail.com and my
> > username is `iamabug`, not sure which one is id.
> >
> > Guozhang Wang  于2020年4月13日周一 下午1:51写道:
> >
> > > The id is for the apache's wiki space:
> > > https://cwiki.apache.org/confluence/display/KAFKA
> > >
> > > If you already had one before, that will work; if not you can create
> one
> > > under that space.
> > >
> > >
> > > Guozhang
> > >
> > > On Sun, Apr 12, 2020 at 10:49 PM 张祥  wrote:
> > >
> > > > I am not sure that I have one, how can I find out this and how can I
> > > create
> > > > one ? Thanks.
> > > >
> > > > Guozhang Wang  于2020年4月13日周一 下午1:42写道:
> > > >
> > > > > Hello Xiang,
> > > > >
> > > > > What's your apache ID?
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Sun, Apr 12, 2020 at 6:08 PM 张祥 
> wrote:
> > > > >
> > > > > > Hi, I am working on a ticket which requires modifying public APIs
> > > that
> > > > > are
> > > > > > visible to users. Could somebody grant the KIP permission to me ?
> > > > Thanks.
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -- Guozhang
>


Re: Permission to create a KIP

2020-04-12 Thread
 I just registered a new account with xiangzhang1...@gmail.com and my
username is `iamabug`, not sure which one is id.

Guozhang Wang  于2020年4月13日周一 下午1:51写道:

> The id is for the apache's wiki space:
> https://cwiki.apache.org/confluence/display/KAFKA
>
> If you already had one before, that will work; if not you can create one
> under that space.
>
>
> Guozhang
>
> On Sun, Apr 12, 2020 at 10:49 PM 张祥  wrote:
>
> > I am not sure that I have one, how can I find out this and how can I
> create
> > one ? Thanks.
> >
> > Guozhang Wang  于2020年4月13日周一 下午1:42写道:
> >
> > > Hello Xiang,
> > >
> > > What's your apache ID?
> > >
> > >
> > > Guozhang
> > >
> > > On Sun, Apr 12, 2020 at 6:08 PM 张祥  wrote:
> > >
> > > > Hi, I am working on a ticket which requires modifying public APIs
> that
> > > are
> > > > visible to users. Could somebody grant the KIP permission to me ?
> > Thanks.
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -- Guozhang
>


Re: Permission to create a KIP

2020-04-12 Thread
I am not sure that I have one, how can I find out this and how can I create
one ? Thanks.

Guozhang Wang  于2020年4月13日周一 下午1:42写道:

> Hello Xiang,
>
> What's your apache ID?
>
>
> Guozhang
>
> On Sun, Apr 12, 2020 at 6:08 PM 张祥  wrote:
>
> > Hi, I am working on a ticket which requires modifying public APIs that
> are
> > visible to users. Could somebody grant the KIP permission to me ? Thanks.
> >
>
>
> --
> -- Guozhang
>


MessagesOutPerSec JMX metric

2020-04-12 Thread
Hi,

I am wondering why there isn't a metric called MessagesOutPerSec in Kafka
JMX metrics to describe how many messages are consumed by clients and
fetched by followers per second since there are already metrics like
MessagesInPerSec, BytesInPerSec and BytesOutPerSec. Thanks.


Permission to create a KIP

2020-04-12 Thread
Hi, I am working on a ticket which requires modifying public APIs that are
visible to users. Could somebody grant the KIP permission to me ? Thanks.


JMX expired topic metircs

2020-03-19 Thread
Hi,

I notice that there are jmx metrics for deleted topics when using java code
and jmxterm. Has anyone also run into this ? If yes, what is the reason
behind this and how can I filter expired metrics ? Thanks.


Re: Regarding segment size config

2020-03-19 Thread
Thanks, it does make sense.

Shailesh Panwar  于2020年3月20日周五 上午12:38写道:

> Hopefully this will help explain the reason behind the 2g limit
>
> https://issues.apache.org/jira/browse/KAFKA-1670?focusedCommentId=14161185=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-14161185
>
>
> On Wed, Mar 18, 2020 at 2:10 AM 张祥  wrote:
>
> > Hi community,
> >
> > I understand that there are two configs regarding segment file size,
> > log.segment.bytes for broker and segment.bytes for topic. The default
> > values are both 1G and they are required to be an integer so they cannot
> > be larger than 2G. My question is, assuming I am not making any mistakes,
> > what is the reason that log segment size is limited below 2G ? Thanks.
> >
>


Kafka JMX monitoring

2020-03-19 Thread
Hi,

I want to know what the best practice to collect Kafka JMX metrics is. I
haven't found a decent way to collect and parse JMX in Java (because it is
too much) and I learn that there are tools like tools like jmxtrans to do
this. I wonder if there is more. Thanks. Regards.


Regarding segment size config

2020-03-18 Thread
Hi community,

I understand that there are two configs regarding segment file size,
log.segment.bytes for broker and segment.bytes for topic. The default
values are both 1G and they are required to be an integer so they cannot
be larger than 2G. My question is, assuming I am not making any mistakes,
what is the reason that log segment size is limited below 2G ? Thanks.


JIRA access request

2020-02-19 Thread
Hi,

I am a newbie, could somebody please open JIRA access for me ? Thanks.