[jira] [Created] (KAFKA-13026) Idempotent producer (KAFKA-10619) follow-up

2021-07-01 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-13026:
-

 Summary: Idempotent producer (KAFKA-10619) follow-up 
 Key: KAFKA-13026
 URL: https://issues.apache.org/jira/browse/KAFKA-13026
 Project: Kafka
  Issue Type: Improvement
Reporter: Cheng Tan


# Adjust config priority
 # Adjust the JUnit tests so we get good coverage of the non-default behavior
 # Similar to 2 for system tests



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [VOTE] KIP-679: Producer will enable the strongest delivery guarantee by default

2020-12-08 Thread Cheng Tan
Thanks you all for discussing and voting on this KIP. Now it has been approved 
with 4 binding votes.

Best, - Cheng

> On Dec 8, 2020, at 12:52 AM, David Jacot  wrote:
> 
> +1 (binding)
> 
> Thanks for the KIP, Cheng!
> 
> On Tue, Dec 8, 2020 at 12:23 AM Ismael Juma  wrote:
> 
>> Thanks, +1 (binding).
>> 
>> On Mon, Dec 7, 2020 at 1:40 PM Cheng Tan  wrote:
>> 
>>> Hi Ismael,
>>> 
>>> Yes. Add deprecation warning for `IDEMPOTENT_WRITE` in 3.0 makes sense.
>>> I’ve updated the KIP’s “DEMPOTENT_WRITE Deprecation” section to reflect
>>> your suggestion. Please let me know if you have more suggestions. Thanks.
>>> 
>>> 
>>> Best, - Cheng Tan
>>> 
>>> 
>>>> On Dec 7, 2020, at 6:42 AM, Ismael Juma  wrote:
>>>> 
>>>> Thanks for the KIP Cheng. One suggestion: should we add the
>> `kafka-acls`
>>>> deprecation warnings for `IDEMPOTENT_WRITE` in 3.0? That would give
>> time
>>>> for authorizer implementations to be updated.
>>>> 
>>>> Ismael
>>>> 
>>>> On Fri, Dec 4, 2020 at 11:00 AM Cheng Tan >> c...@confluent.io>> wrote:
>>>> 
>>>>> Hi all,
>>>>> 
>>>>> I’m proposing a new KIP for enabling the strongest delivery guarantee
>> by
>>>>> default. Today Kafka support EOS and N-1 concurrent failure tolerance
>>> but
>>>>> the default settings haven’t bring them out of the box. The proposal
>>>>> changes include the producer defaults change to `ack=all` and
>>>>> `enable.idempotence=true`. Also, the ACL operation type
>>> IDEMPOTENCE_WRITE
>>>>> will be deprecated. If a producer has WRITE permission to any topic,
>> it
>>>>> will be able to request a producer id and perform idempotent produce.
>>>>> 
>>>>> KIP here:
>>>>> 
>>>>> 
>>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default
>>>>> <
>>>>> 
>>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-679:+Producer+will+enable+the+strongest+delivery+guarantee+by+default
>>> <
>>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-679:+Producer+will+enable+the+strongest+delivery+guarantee+by+default
>>>> 
>>>>>> 
>>>>> 
>>>>> Please vote in this mail thread.
>>>>> 
>>>>> Thanks
>>>>> 
>>>>> - Cheng Tan
>>> 
>>> 
>> 



Re: [VOTE] KIP-679: Producer will enable the strongest delivery guarantee by default

2020-12-07 Thread Cheng Tan
Hi Ismael,

Yes. Add deprecation warning for `IDEMPOTENT_WRITE` in 3.0 makes sense. I’ve 
updated the KIP’s “DEMPOTENT_WRITE Deprecation” section to reflect your 
suggestion. Please let me know if you have more suggestions. Thanks.


Best, - Cheng Tan


> On Dec 7, 2020, at 6:42 AM, Ismael Juma  wrote:
> 
> Thanks for the KIP Cheng. One suggestion: should we add the `kafka-acls`
> deprecation warnings for `IDEMPOTENT_WRITE` in 3.0? That would give time
> for authorizer implementations to be updated.
> 
> Ismael
> 
> On Fri, Dec 4, 2020 at 11:00 AM Cheng Tan  <mailto:c...@confluent.io>> wrote:
> 
>> Hi all,
>> 
>> I’m proposing a new KIP for enabling the strongest delivery guarantee by
>> default. Today Kafka support EOS and N-1 concurrent failure tolerance but
>> the default settings haven’t bring them out of the box. The proposal
>> changes include the producer defaults change to `ack=all` and
>> `enable.idempotence=true`. Also, the ACL operation type IDEMPOTENCE_WRITE
>> will be deprecated. If a producer has WRITE permission to any topic, it
>> will be able to request a producer id and perform idempotent produce.
>> 
>> KIP here:
>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default
>> <
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-679:+Producer+will+enable+the+strongest+delivery+guarantee+by+default
>>  
>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-679:+Producer+will+enable+the+strongest+delivery+guarantee+by+default>
>>> 
>> 
>> Please vote in this mail thread.
>> 
>> Thanks
>> 
>> - Cheng Tan



[VOTE] KIP-679: Producer will enable the strongest delivery guarantee by default

2020-12-04 Thread Cheng Tan
Hi all,

I’m proposing a new KIP for enabling the strongest delivery guarantee by 
default. Today Kafka support EOS and N-1 concurrent failure tolerance but the 
default settings haven’t bring them out of the box. The proposal changes 
include the producer defaults change to `ack=all` and 
`enable.idempotence=true`. Also, the ACL operation type IDEMPOTENCE_WRITE will 
be deprecated. If a producer has WRITE permission to any topic, it will be able 
to request a producer id and perform idempotent produce.

KIP here:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default
 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-679:+Producer+will+enable+the+strongest+delivery+guarantee+by+default>

Please vote in this mail thread.

Thanks

- Cheng Tan

Re: Support of virtual threads by Kafka

2020-11-11 Thread Cheng Tan
One thing I can think of is the ACL operations (create / delete ACLs). They are 
much more less frequently used other than the produce/consume/topic creation so 
I’m not sure if it’s valuable to use as a benchmark.

- Cheng 

> On Nov 11, 2020, at 4:33 PM, Leonid Mesnik  wrote:
> 
> Hi Ismael
> 
> Thank you for your help and quick response. You are right, the file IO is 
> blocking  with Loom now. And small number threads might not having benefits 
> if they were just changed to virtual threads.
> 
> The virtual threads could help when you have a lot of tasks which are 
> executed in some thread pool.  So each such task is executed in separate 
> virtual thread and could be yielded instead of blocking thread. Do you think 
> you have something similar in Kafka?
> 
> It would be great if you just point me to such tasks. So I could try to 
> execute them in virtual thread pool. If you don't think you have them or they 
> couldn't be easily "virtualized" it is fine. I understand that Loom is still 
> a separate project and it might be premature to think about any significant 
> changes related to it. So now I would like just to try to use such changes to 
> test Loom implementation itself.
> 
> Leonid
> 
> On 11/10/20 6:24 AM, Ismael Juma wrote:
>> Hi Leonid,
>> 
>> Thanks for looking into this. I think the main question is how
>> virtual threads would help here. Kafka tends to use a small number of
>> threads combined with non blocking IO, but there are some exceptions like
>> disk IO. My understanding is that the latter is still blocking even with
>> the latest builds of Loom. Is that right?
>> 
>> Ismael
>> 
>> On Mon, Nov 9, 2020 at 10:26 AM Leonid Mesnik 
>> wrote:
>> 
>>> Hi
>>> 
>>> 
>>> Currently, I am working on Loom project which enables virtual threads
>>> for Java.
>>> 
>>> https://wiki.openjdk.java.net/display/loom/Main
>>> 
>>> As one of real-life tests it would be interesting to run Kafka using
>>> virtual threads. However it might require some support in Kafka for
>>> this. It is needed to add ability to start some threads as "virtual". Do
>>> you know if anyone is interested and could help me with this?
>>> 
>>> Here are more details:
>>> 
>>> Basically, the virtual thread is a sub-class of java.lang.Thread. So it
>>> is need to refactor code to avoid subclassing of thread and factorize
>>> thread creation. I placed "example" fix what should be done to add
>>> ability to run KafkaThread as virtual thread. It is just to demonstrate
>>> the overall idea of changes.
>>> 
>>> 
>>> https://urldefense.com/v3/__https://github.com/lmesnik/kafka/commit/872a2d5fd57b0c76878eece6c54c783897ccbf5e__;!!GqivPVa7Brio!MaLCXTeiTLzVpWU8BD3j4o6RxBqSOki307AHuhzHGrX1Q_9khpAC-9FTvrXyrcGUNA$
>>>  
>>> 
>>> 
>>> 
>>> I want to check with you if it is a good approach for Kafka and are
>>> there are other places to be updated. There is no plan to push such
>>> support in mainline yet. Also, no plans to make any significant changes.
>>> But if they want we could do it.
>>> 
>>> What do you think about this?
>>> 
>>> Leonid



Re: [DISCUSS] KIP-679 Producer will enable the strongest delivery guarantee by default

2020-11-04 Thread Cheng Tan
I made the following changes since I sent out the last discussion message:

1. Rename org.apache.kafka.server.authorizer.Authorizer#authorizeAny to 
org.apache.kafka.server.authorizer.Authorizer#authorizeByResourceType
2. Optimized the interface default of 
org.apache.kafka.server.authorizer.Authorizer#authorizeAny, now it’s not 
coupled with org.apache.kafka.server.authorizer.Authorizer#authorize any more 
and having a better performance.

Please feel free to comment and leave any thoughts. Any feedback will be 
appreciated. Thanks.

Best, - Cheng 

> On Oct 19, 2020, at 9:15 PM, Cheng Tan  wrote:
> 
> Hi all,
> 
> I’m proposing a new KIP for enabling the strongest delivery guarantee by 
> default. Today Kafka support EOS and N-1 concurrent failure tolerance but the 
> default settings haven’t bring them out of the box. The proposal is 
> discussing the best approach to change the producer defaults to `ack=all` and 
> `enable.idempotence=true`. Please join the discussion here: 
> 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default
>  
> <https://cwiki.apache.org/confluence/display/KAFKA/KIP-679:+Producer+will+enable+the+strongest+delivery+guarantee+by+default>
> 
> Thanks
> 
> - Cheng Tan



[DISCUSS] KIP-679 Producer will enable the strongest delivery guarantee by default

2020-10-19 Thread Cheng Tan
Hi all,

I’m proposing a new KIP for enabling the strongest delivery guarantee by 
default. Today Kafka support EOS and N-1 concurrent failure tolerance but the 
default settings haven’t bring them out of the box. The proposal is discussing 
the best approach to change the producer defaults to `ack=all` and 
`enable.idempotence=true`. Please join the discussion here: 

https://cwiki.apache.org/confluence/display/KAFKA/KIP-679%3A+Producer+will+enable+the+strongest+delivery+guarantee+by+default
 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-679:+Producer+will+enable+the+strongest+delivery+guarantee+by+default>

Thanks

- Cheng Tan

[jira] [Created] (KAFKA-10619) Producer will enable EOS by default

2020-10-19 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-10619:
-

 Summary: Producer will enable EOS by default
 Key: KAFKA-10619
 URL: https://issues.apache.org/jira/browse/KAFKA-10619
 Project: Kafka
  Issue Type: Improvement
Reporter: Cheng Tan
Assignee: Cheng Tan


This is an after-work for KIP-185. 

In the producer config,
 # the default value of `acks` will change to `all`
 # `enable.idempotence` will change to `true`

[An analysis of the impact of max.in.flight.requests.per.connection and acks on 
Producer 
performance|https://cwiki.apache.org/confluence/display/KAFKA/An+analysis+of+the+impact+of+max.in.flight.requests.per.connection+and+acks+on+Producer+performance]
 indicates that changing `acks` from `1` to `all` won't increase the latency 
and decrease the throughput in a significant way.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [DISCUSSION] KIP-619: Add internal topic creation support

2020-08-19 Thread Cheng Tan
Hi David,


Thanks for the feedback. They are really helpful.

> Can you clarify a bit more what the difference is between regular topics
> and internal topics (excluding  __consumer_offsets and
> __transaction_state)? Reading your last message, if internal topics
> (excluding the two) can be created, deleted, produced to, consumed from,
> added to transactions, I'm failing to see what is different about them. Is
> it simply that they are marked as "internal" so the application can treat
> them differently?

Yes. The user-defined internal topics (those except `__consumer_offsets` and 
`__transaction_state`) will behave as normal topics in regard to messaging 
operation and permission. Topics are marked as “internal” in order to make the 
broker able to test user-defined internal topics and better provide metadata 
services, such as `listTopics` API. I should have added the metadata behavior 
difference in the KIP.

> In the "Compatibility, Deprecation, and Migration" section, we should
> detail how users can overcome this incompatibility (i.e., changing the
> config name on their topic and changing their application logic if
> necessary).

Thanks for the suggestion. I updated the section.

> Should we consider adding any configs to constrain the min isr and
> replication factor for internal topics? If a topic is really internal and
> fundamentally required for an application to function, it might need a more
> stringent replication config. Our existing internal topics have their own
> configs in server.properties with a comment saying as much.


I think we should probably give clients the freedom to configure 
`min.insync.replicas`, `replication.factor`, and `log.retention` on 
user-defined internal topics as they do on normal topics.

1. Users may have performance requirements on user-defined internal topics.
2. Potential new defaults / restrictions may change the existing user 
application logic silently. There might be compatibility issues.
3. Since user-defined internal topics act like normal topics and won’t affect 
the messaging functionality (produce, consume, transaction, etc), unoptimized 
log configurations won’t harm the cluster. 


Please let me know what you think. Thanks.


Best, - Cheng Tan



> On Aug 14, 2020, at 7:44 AM, David Arthur  wrote:
> 
> Cheng,
> 
> Can you clarify a bit more what the difference is between regular topics
> and internal topics (excluding  __consumer_offsets and
> __transaction_state)? Reading your last message, if internal topics
> (excluding the two) can be created, deleted, produced to, consumed from,
> added to transactions, I'm failing to see what is different about them. Is
> it simply that they are marked as "internal" so the application can treat
> them differently?
> 
> 
> In the "Compatibility, Deprecation, and Migration" section, we should
> detail how users can overcome this incompatibility (i.e., changing the
> config name on their topic and changing their application logic if
> necessary).
> 
> 
> Should we consider adding any configs to constrain the min isr and
> replication factor for internal topics? If a topic is really internal and
> fundamentally required for an application to function, it might need a more
> stringent replication config. Our existing internal topics have their own
> configs in server.properties with a comment saying as much.
> 
> 
> Thanks!
> David
> 
> 
> 
> On Tue, Jul 7, 2020 at 1:40 PM Cheng Tan  wrote:
> 
>> Hi Colin,
>> 
>> 
>> Thanks for the comments. I’ve modified the KIP accordingly.
>> 
>>> I think we need to understand which of these limitations we will carry
>> forward and which we will not.  We also have the option of putting
>> limitations just on consumer offsets, but not on other internal topics.
>> 
>> 
>> In the proposal, I added details about this. I agree that cluster admin
>> should use ACLs to apply the restrictions.
>> Internal topic creation will be allowed.
>> Internal topic deletion will be allowed except for` __consumer_offsets`
>> and `__transaction_state`.
>> Producing to internal topic partitions other than `__consumer_offsets` and
>> `__transaction_state` will be allowed.
>> Adding internal topic partitions to transactions will be allowed.
>>> I think there are a fair number of compatibility concerns.  What's the
>> result if someone tries to create a topic with the configuration internal =
>> true right now?  Does it fail?  If not, that seems like a potential problem.
>> 
>> I also added this compatibility issue in the "Compatibility, Deprecation,
>> and Migration Plan" section.
>> 
>> Please feel free to make any suggestions o

Re: [DISCUSSION] KIP-619: Add internal topic creation support

2020-07-07 Thread Cheng Tan
Hi Colin,


Thanks for the comments. I’ve modified the KIP accordingly.

> I think we need to understand which of these limitations we will carry 
> forward and which we will not.  We also have the option of putting 
> limitations just on consumer offsets, but not on other internal topics.


In the proposal, I added details about this. I agree that cluster admin should 
use ACLs to apply the restrictions. 
Internal topic creation will be allowed.
Internal topic deletion will be allowed except for` __consumer_offsets` and 
`__transaction_state`.
Producing to internal topic partitions other than `__consumer_offsets` and 
`__transaction_state` will be allowed.
Adding internal topic partitions to transactions will be allowed.
> I think there are a fair number of compatibility concerns.  What's the result 
> if someone tries to create a topic with the configuration internal = true 
> right now?  Does it fail?  If not, that seems like a potential problem.

I also added this compatibility issue in the "Compatibility, Deprecation, and 
Migration Plan" section.

Please feel free to make any suggestions or comments regarding to my latest 
proposal. Thanks.


Best, - Cheng Tan






> On Jun 15, 2020, at 11:18 AM, Colin McCabe  wrote:
> 
> Hi Cheng,
> 
> The link from the main KIP page is an "edit link" meaning that it drops you 
> into the editor for the wiki page.  I think the link you meant to use is a 
> "view link" that will just take you to view the page.
> 
> In general I'm not sure what I'm supposed to take away from the large UML 
> diagram in the KIP.  This is just a description of the existing code, right?  
> Seems like we should remove this.
> 
> I'm not sure why the controller classes are featured here since as far as I 
> can tell, the controller doesn't need to care if a topic is internal.
> 
>> Kafka and its upstream applications treat internal topics differently from
>> non-internal topics. For example:
>> * Kafka handles topic creation response errors differently for internal 
>> topics
>> * Internal topic partitions cannot be added to a transaction
>> * Internal topic records cannot be deleted
>> * Appending to internal topics might get rejected
> 
> I think we need to understand which of these limitations we will carry 
> forward and which we will not.  We also have the option of putting 
> limitations just on consumer offsets, but not on other internal topics.
> 
> Taking it one by one:
> 
>> * Kafka handles topic creation response errors differently for internal 
>> topics.
> 
> Hmm.  Kafka doesn't currently allow you to create internal topics, so the 
> difference here is that you always fail, right?  Or is there something else 
> more subtle here?  Like do we specifically prevent you from creating topics 
> named __consumer_offsets or something?  We need to spell this all out in the 
> KIP.
> 
>> * Internal topic partitions cannot be added to a transaction
> 
> I don't think we should carry this limitation forward, or if we do, we should 
> only do it for consumer-offsets.  Does anyone know why this limitation exists?
> 
>> * Internal topic records cannot be deleted
> 
> This seems like something that should be handled by ACLs rather than by 
> treating internal topics specially.
> 
>> * Appending to internal topics might get rejected
> 
> We clearly need to use ACLs here rather than rejecting appends.  Otherwise, 
> how will external systems like KSQL, streams, etc. use this feature?  This is 
> the kind of information we need to have in the KIP.
> 
>> Public Interfaces
>> 2. KafkaZkClient will have a new method getInternalTopics() which 
>> returns a set of internal topic name strings.
> 
> KafkaZkClient isn't a public interface, so it doesn't need to be described 
> here.
> 
>> There are no compatibility concerns in this KIP.
> 
> I think there are a fair number of compatibility concerns.  What's the result 
> if someone tries to create a topic with the configuration internal = true 
> right now?  Does it fail?  If not, that seems like a potential problem.
> 
> Are people going to be able to create or delete topics named 
> __consumer_offsets or __transaction_state using this mechanism?  If so, how 
> does the security model work for that?
> 
> best,
> Colin
> 
> On Fri, May 29, 2020, at 01:09, Cheng Tan wrote:
>> Hello developers,
>> 
>> 
>> I’m proposing KIP-619 to add internal topic creation support. 
>> 
>> Kafka and its upstream applications treat internal topics differently 
>> from non-internal topics. For example:
>> 
>>  • Kafka handles topic creation

Re: [VOTE] KIP-601: Configurable socket connection timeout in NetworkClient

2020-06-05 Thread Cheng Tan
The KIP is approved upon 3 binding votes. Thanks for all the feedback and votes!

- Cheng

Re: [VOTE] KIP-601: Configurable socket connection timeout in NetworkClient

2020-06-03 Thread Cheng Tan
Dear Rajini,

Thanks for the feedback. 

1) 
Because "request.timeout.ms" only affects in-flight requests, after the API 
NetworkClient.ready() is invoked, the connection won't get closed after 
"request.timeout.ms” hits. Before 
a) the SocketChannel is connected
b) ssl handshake finished
c) authentication has finished (sasl) 
clients cannot invoke NetworkClient.send() to send any request, which means no 
in-flight request targeting to the connection will be added.


2) 
I think a default value of 127 seconds make sense, which meets the timeout 
indirectly specified by the default value of “tcp.syn.retries”. I’ve added this 
into the KIP proposal.


3) 
Every time the timeout hits, the timeout value of the next connection try will 
increase. 

The timeout will hit iff a connection stays at the `connecting` state longer 
than the timeout value, as indicated by 
ClusterConnectionStates.NodeConnectionState. The connection state of a node 
will change iff `SelectionKey.OP_CONNECT` is detected by 
`nioSelector.Select()`. The connection state may transit from `connecting` to 

a) `disconnected` when SocketChannel.finishConnect() throws IOException.
b) `connected` when SocketChannel.finishConnect() return TRUE.

In other words, the timeout will hit and increase iff the interested 
SelectionKey.OP_CONNECT doesn't trigger before the timeout arrives, which 
means, for example, network congestion, failure of the ARP request, packet 
filtering, routing error, or a silent discard may happen. (I didn’t read the 
Java NIO source code. Please correct me the case when OP_CONNECT won’t get 
triggered if I’m wrong)


4) 

A) Connection timeout dominates both request timeout and API timeout

When connection timeout hits, the connection will be closed. The client will be 
notified either by the responses constructed by NetworkClient or the callbacks 
attached to the request. As a result, the request failure will be handled 
before either connection timeout or API timeout arrives.


B) Neither request timeout or API timeout dominates connection timeout

i) Request timeout: Because request timeout only affects in-flight requests, 
after the API NetworkClient.ready() is invoked, the connection won't get closed 
after "request.timeout.ms” hits. Before
1. the SocketChannel is connected
2. SSL handshake finished
3. authentication has finished (SASL)
, clients won't be able to invoke NetworkClient.send() to send any request, 
which means no in-flight request targeting to the connection will be added.

ii) API timeout: In AdminClient, API timeout acts by putting a smaller and 
smaller timeout value to the chain of requests in a same API. After the API 
timeout hits, the retry logic won't close any connection. In consumer, API 
timeout acts as a whole by putting a limit to the code block executing time. 
The retry logic won't close any connection as well.


Conclusion: 

Thanks again for the long feedback and I’m always enjoying them. I’ve 
supplement the above discussion into the KIP proposal. Please let me know what 
you think.


Best, - Cheng Tan


> On Jun 2, 2020, at 3:01 AM, Rajini Sivaram  wrote:
> 
> Hi Cheng,
> 
> Not sure if the discussion should move back to the DISCUSS thread. I have a
> few questions:
> 
> 1) The KIP motivation says that in some cases `request.timeout.ms` doesn't
> timeout connections properly and as a result it takes 127s to detect a
> connection failure. This sounds like a bug rather than a limitation of the
> current approach. Can you explain the scenarios where this occurs?
> 
> 2) I think the current proposal is to use non-exponential 10s connection
> timeout as default with the option to use exponential timeout. So
> connection timeouts for every connection attempt will be between 8s and 12s
> by default. Is that correct? Should we use a default max timeout to enable
> exponential timeout by default since 8s seems rather small?
> 
> 3) What is the scope of `failures` used to determine connection timeout
> with exponential timeouts? Will we always use 10s followed by 20s every
> time a connection is attempted?
> 
> 4) It will be good if we can include two flows with the relationship
> between various timeouts in the KIP. One with a fixed node like a typical
> produce/consume request to the leader and another that uses
> `leastLoadedNode` like a metadata request. Having the comparison between
> the current and proposed behaviour w.r.t all configurable timeouts (the two
> new connection timeouts, request timeout, api timeout etc.) will be useful.
> 
> Regards,
> 
> Rajini
> 


Re: [DISCUSS] KIP-619 Deprecate ConsumerConfig#addDeserializerToConfig(Properties, Deserializer, Deserializer) and ProducerConfig#addSerializerToConfig(Properties, Serializer, Serializer)

2020-06-02 Thread Cheng Tan
Hi Chia, 

Hope you are doing well. I already took KIP-619 as my KIP identification 
number. Could you change your KIP id? Thank you.

Best, - Cheng

> On May 31, 2020, at 8:08 PM, Chia-Ping Tsai  wrote:
> 
> hi All,
> 
> This KIP plans to deprecate two unused methods without replacement.
> 
> All suggestions are welcome!
> 
> KIP: 
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=155749118
> ISSUE: https://issues.apache.org/jira/browse/KAFKA-10044
> 
> ---
> Chia-Ping



[DISCUSSION] KIP-619: Add internal topic creation support

2020-05-29 Thread Cheng Tan
Hello developers,


I’m proposing KIP-619 to add internal topic creation support. 

Kafka and its upstream applications treat internal topics differently from 
non-internal topics. For example:

• Kafka handles topic creation response errors differently for internal 
topics
• Internal topic partitions cannot be added to a transaction
• Internal topic records cannot be deleted
• Appending to internal topics might get rejected
• ……

Clients and upstream applications may define their own internal topics. For 
example, Kafka Connect defines `connect-configs`, `connect-offsets`, and 
`connect-statuses`. Clients are fetching the internal topics by sending the 
MetadataRequest (ApiKeys.METADATA).

However, clients and upstream application cannot register their own internal 
topics in servers. As a result, servers have no knowledge about client-defined 
internal topics. They can only test if a given topic is internal or not simply 
by checking against a static set of internal topic string, which consists of 
two internal topic names `__consumer_offsets` and `__transaction_state`. As a 
result, MetadataRequest cannot provide any information about client created 
internal topics. 

To solve this pain point, I'm proposing support for clients to register and 
query their own internal topics. 

Please feel free to join the discussion. Thanks in advance.


Best, - Cheng Tan

Re: [VOTE] KIP-601: Configurable socket connection timeout in NetworkClient

2020-05-19 Thread Cheng Tan
Dear Colin,


Thanks for the reply. Your reasoning make sense. I’ve modified the KIP-601 with 
two changes:

1. Now KIP-601 is proposing a exponential connection setup timeout, which is 
controlled by socket.connections.setup.timeout.ms (init value) and 
socket.connections.setup.timeout.max.ms (max value)

2. The logic optimization in leastLoadedNode(), which I want to discuss on that 
again. In the scenario that no connected or connection node exists, instead of 
providing the node with least failed attempts, the NetworkClient can provide 
the least recently used node which respects the reconnect backoff. The existing 
property ClusterConnectionStates.NodeConnectionState.lastConnectAttemptMs can 
help us pick the LRU node conveniently. Does this make sense to you?

Please let me know what you think. Thanks.


Best, - Cheng Tan



> On May 19, 2020, at 1:44 PM, Colin McCabe  wrote:
> 
> It seems like this analysis is assuming that the only reason to wait longer 
> is so that we can send another SYN packet.  This may not be the case-- 
> perhaps waiting longer would allow us to receive an ACK from the remote end 
> that has been delayed for some reason while going through the network.
> 
> We also probably don't want our expiration time period to line up exactly 
> with Linux's retries.  If it did, we would cut off the connection attempt 
> just as we were re-sending another SYN.
> 
> Also, there are other OSes besides Linux, and other configurations besides 
> the default one.
> 
> So, on the whole, I don't think we need to make the default a power of 2.
> 
> best,
> Colin
> 



Re: [VOTE] KIP-601: Configurable socket connection timeout in NetworkClient

2020-05-18 Thread Cheng Tan
Dear Colin,


Thanks for the suggestions.

> For example, if a new node joins the cluster, it will have 0 failed connect 
> attempts, whereas the existing nodes will probably have more than 0.  So all 
> the clients will ignore every other node and pile on to the new one.  That's 
> not good


The existing behavior is not random when there’s no connected or connected 
node. leastLoadeNode() will always provide the node respect the connection 
backoff with the largest array index in the cached node list. The shuffle only 
happens after metadata fetch. Thus, when the client is not able to fetch 
metadata, the cached node won’t get shuffled. So I proposed to consider the 
failed attempts together with the connection backoff. 

The potential issue you mentioned make sense. I can think about an alternative 
way which is to randomly pick a disconnected node which respect the connection 
backoff.

> Consider the case where we need to talk to the controller but it is not 
> responding.  With the current proposal we will keep trying to reconnect every 
> 10 seconds.  That could lead to more reconnection attempts than what happens 
> today.  In the rare case where the node is taking more than 10 seconds to 
> process new connections, it will prevent us from connecting completely.

Exponential timeout make sense. I also have some thoughts about the parameter 
tuning. Since Java NIO will timeout and retry the socket channel connection 
exponentially after 1s, 2s, 4s, 8s, …, we’d better to make the default value as 
a exp of 2 since the sum of the timeout by Java NIO is 2^x  - 1. 

For example, if the socket.connection.setup.timeout = 10, Java NIO will only 
get a chance to try a maximum timeout 4 since 1 + 2 + 4 = 7 and the last try is 
less than 3s, which is useless. However, if we set the 
socket.connection.setup.timeout = 8 or 16, the last try won’t get wasted since 
1 + 2 + 4 = 7 and 1 + 2 + 4 + 8 = 15.


Please let me know what you think. Thanks.

Best, - Cheng Tan



> On May 18, 2020, at 1:32 PM, Colin McCabe  wrote:
> 
> Hi Cheng,
> 
> socket.connection.setup.timeout.ms seems more consistent with our existing 
> configuration names than socket.connections.setup.timeout.ms (with an s).  
> What do you think?
> 
>> If no connected or connecting node exists, provide the disconnected node 
>> which
>> respects the reconnect backoff with the least number of failed attempts.
> 
> I think we need to rethink this part.  For example, if a new node joins the 
> cluster, it will have 0 failed connect attempts, whereas the existing nodes 
> will probably have more than 0.  So all the clients will ignore every other 
> node and pile on to the new one.  That's not good.  I think we should just 
> keep the existing random behavior.  If the node isn't blacklisted due to 
> connection backoff, it should be fair game to be connected to.
> 
> On a related note, I think it would be good to have an exponential connection 
> setup timeout backoff, similar to what we do with reconnect backoff.
> 
> Consider the case where we need to talk to the controller but it is not 
> responding.  With the current proposal we will keep trying to reconnect every 
> 10 seconds.  That could lead to more reconnection attempts than what happens 
> today.  In the rare case where the node is taking more than 10 seconds to 
> process new connections, it will prevent us from connecting completely.
> 
> An exponential strategy could start at 10 seconds, then do 20, then 40, then 
> 80, up to some limit.  That would reduce the extra load and also handle the 
> (hopefully very rare) case where connections are taking a long time to 
> connect.
> 
> best,
> Colin
> 



Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-15 Thread Cheng Tan
Dear Rajini,

Thanks for all the feedbacks. They are very helpful for me to do the 
brainstorming. I’ve incorporated our discuss in the KIP and started a voting 
thread. 

Best, - Cheng Tan

> On May 15, 2020, at 2:13 PM, Rajini Sivaram  wrote:
> 
> Hi Cheng,
> 
> I am fine with the rest of the KIP apart from the 10s default. If no one
> else has any concerns about this new default, let's go with it. Please go
> ahead and start vote.
> 
> Regards,
> 
> Rajini
> 
> 
> On Fri, May 15, 2020 at 8:21 PM Cheng Tan  wrote:
> 
>> Dear Rajini,
>> 
>> 
>> Thanks for the reply.
>> 
>>> e have a lot of these and I want to
>>> understand the benefits of the proposed timeout in this case alone. We
>>> currently have a request timeout of 30s. Would you consider adding a 10s
>>> connection timeout?
>> 
>> A shorter timeout (10s) at the transportation level will help clients
>> detect dead nodes faster. “request.timeout.ms” is too general and applies
>> to all the requests whose complexity at the application level varies.  It’s
>> risky to set “request.timeout.ms” to a lower value for detecting dead
>> nodes quicker because of the involvement of the application layer.
>> 
>> After “socket.connection.setup.timeout.ms” hits, NetworkClient will fail
>> the request in the exact approach as it handles “request.timeout.ms”.
>> That is to say, the response will constructed upon a RetriableException.
>> Producer, Consumer, and KafkaAdminClient can then perform their retry logic
>> as a request timeout happens.
>> 
>>> We have KIP-612 that is proposing to throttle connection set up on the
>> one
>>> hand and this KIP that is dramatically reducing default connection
>> timeout
>>> on the other. Not sure if that is a good idea.
>> 
>> The default of the broker connection creation rate limit is Int.MaxValue.
>> The KIP also proposes per-IP throttle configuration. Thus, I don’t expect
>> the combination of the broker connection throttle and a shorter client
>> transportation timeout will have a side effect.
>> 
>> Does the reasons above make sense to you?
>> 
>> Best, - Cheng
>> 
>> 
>> 
>> 
>>> On May 15, 2020, at 4:49 AM, Rajini Sivaram 
>> wrote:
>>> 
>>> Hi Cheng,
>>> 
>>> Let me rephrase my question. Let's say we didn't have the case of
>>> leastLoadedNode. We are only talking about connections to a specific node
>>> (i.e. leader or controller). We have a lot of these and I want to
>>> understand the benefits of the proposed timeout in this case alone. We
>>> currently have a request timeout of 30s. Would you consider adding a 10s
>>> connection timeout? And if you did, what would you expect the 10s timeout
>>> to do?
>>> 
>>> a) We could fail a request if connection didn't complete within 10s. If
>> we
>>> always expect connections to succeed within 10s, this would be considered
>>> reasonable behaviour. But this would be changing the current default,
>> which
>>> allows you up to 30 seconds to connect and process a request.
>>> b) We retry the connection. What would be the point? We were waiting in a
>>> queue for connecting, but we decide to stop and join the back of the
>> queue.
>>> 
>>> We have KIP-612 that is proposing to throttle connection set up on the
>> one
>>> hand and this KIP that is dramatically reducing default connection
>> timeout
>>> on the other. Not sure if that is a good idea.
>>> 
>>> 
>>> On Fri, May 15, 2020 at 1:26 AM Cheng Tan  wrote:
>> 
>> 



[VOTE] KIP-601: Configurable socket connection timeout in NetworkClient

2020-05-15 Thread Cheng Tan
Hello developers,

Big thanks for all the feedbacks. KIP-601 is finalized and ready for a vote. 

https://cwiki.apache.org/confluence/display/KAFKA/KIP-601%3A+Configurable+socket+connection+timeout+in+NetworkClient
 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-601:+Configurable+socket+connection+timeout+in+NetworkClient>

Best, - Cheng Tan

Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-15 Thread Cheng Tan
Dear Rajini,


Thanks for the reply. 

> e have a lot of these and I want to
> understand the benefits of the proposed timeout in this case alone. We
> currently have a request timeout of 30s. Would you consider adding a 10s
> connection timeout? 

A shorter timeout (10s) at the transportation level will help clients detect 
dead nodes faster. “request.timeout.ms” is too general and applies to all the 
requests whose complexity at the application level varies.  It’s risky to set 
“request.timeout.ms” to a lower value for detecting dead nodes quicker because 
of the involvement of the application layer.

After “socket.connection.setup.timeout.ms” hits, NetworkClient will fail the 
request in the exact approach as it handles “request.timeout.ms”. That is to 
say, the response will constructed upon a RetriableException. Producer, 
Consumer, and KafkaAdminClient can then perform their retry logic as a request 
timeout happens.

> We have KIP-612 that is proposing to throttle connection set up on the one
> hand and this KIP that is dramatically reducing default connection timeout
> on the other. Not sure if that is a good idea.

The default of the broker connection creation rate limit is Int.MaxValue. The 
KIP also proposes per-IP throttle configuration. Thus, I don’t expect the 
combination of the broker connection throttle and a shorter client 
transportation timeout will have a side effect.

Does the reasons above make sense to you? 

Best, - Cheng




> On May 15, 2020, at 4:49 AM, Rajini Sivaram  wrote:
> 
> Hi Cheng,
> 
> Let me rephrase my question. Let's say we didn't have the case of
> leastLoadedNode. We are only talking about connections to a specific node
> (i.e. leader or controller). We have a lot of these and I want to
> understand the benefits of the proposed timeout in this case alone. We
> currently have a request timeout of 30s. Would you consider adding a 10s
> connection timeout? And if you did, what would you expect the 10s timeout
> to do?
> 
> a) We could fail a request if connection didn't complete within 10s. If we
> always expect connections to succeed within 10s, this would be considered
> reasonable behaviour. But this would be changing the current default, which
> allows you up to 30 seconds to connect and process a request.
> b) We retry the connection. What would be the point? We were waiting in a
> queue for connecting, but we decide to stop and join the back of the queue.
> 
> We have KIP-612 that is proposing to throttle connection set up on the one
> hand and this KIP that is dramatically reducing default connection timeout
> on the other. Not sure if that is a good idea.
> 
> 
> On Fri, May 15, 2020 at 1:26 AM Cheng Tan  wrote:



Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-14 Thread Cheng Tan
Hi Rajini,

Thanks for the reply. 

> Not sure 10s is a good default because it unnecessarily times out
> connections, only to attempt re-connecting to the same broker (except in
> the leastLoadedNode case where it would be useful to have a lower timeout).


The underlying logic for a connection turn from “connecting” to “connected” is 
finishing the TCP three-way handshake (done by socketChannel.connect()). So we 
can say the 10 seconds timeout is for the transportation layer three-way 
handshake.

The request timeout includes more besides the handshakes, such as 
authentication, server processing the request, and server sending back the 
response. 

Thus I think setting the default of socket.connection.setup.timeout.ms a 
smaller value than request.timeout.ms is reasonable. Do you have any specific 
reason in mind that 10s is too short? 

Best, -Chang Tan 


> 在 2020年5月14日,上午6:44,Rajini Sivaram  写道:
> 
> Hi Cheng,
> 
> 1) Thanks for the update,  the KIP now says `
> socket.connections.setup.timeout.ms*`*, which sounds good.
> 
> 2) Not sure 10s is a good default because it unnecessarily times out
> connections, only to attempt re-connecting to the same broker (except in
> the leastLoadedNode case where it would be useful to have a lower timeout).
> Couldn't we just use 30s (request timeout) as the default value, retaining
> the current behaviour? Or alternatively, only apply the timeout where it
> makes sense to have a lower timeout (i.e. a timeout for leastLoadedNode)?
> 
> Regards,
> 
> Rajini
> 
>> On Thu, May 14, 2020 at 5:00 AM Cheng Tan  wrote:
>> 
>> Hi Rajini,
>> 
>> Thanks for the comments.
>> 
>>> I think
>>> they started off as connection timeouts but now include authentication
>> time
>>> as well. Have we considered using similar configs for this case?
>> 
>> 
>> The new config I proposed is focusing on the connections to unreachable
>> servers. The timeout count won’t involved the authentication. I think
>> “socket.” prefix make sense. I’ll change it. Colin mentioned that adding
>> the key word “setup” can help people understand that this config only
>> applies to the connection setup. What about “socket.connection.setup.ms”?
>> 
>>> The KIP proposes 10s as the default. What does this mean for typical
>>> connections like a produce request going to the leader?
>> 
>> The new timeout config applies to each connection. It’s at the
>> NetworkClient level and won’t consider the underlying connection logic.
>> Specifically, by default, every connection will have 10 seconds to become
>> “connected” from “connecting”, which implies the corresponding socket
>> channel is now connected (SocketChanel.finishConnect() returns True), no
>> matter what the request logic and abstraction is.
>> 
>> Please let me know if these make sense to you or if you have more
>> thoughts. Thank you.
>> 
>> Best, -Cheng
>> 
>>>> 在 2020年5月13日,上午7:11,Rajini Sivaram  写道:
>>> 
>>> Hi Cheng,
>>> 
>>> Thanks for the KIP, sounds like a good improvement. A couple of comments:
>>> 
>>> 1) We currently have client connection timeouts on the broker with
>> configs
>>> named `xxx.socket.timeout.ms` (e.g. controller.socket.timeout.ms). I
>> think
>>> they started off as connection timeouts but now include authentication
>> time
>>> as well. Have we considered using similar configs for this case? We may
>>> want to prefix the new config with `socket.` anyway - something along the
>>> lines of `socket.connection.timeout.ms` if it is just the connection
>> time.
>>> 
>>> 2) The KIP proposes 10s as the default. What does this mean for typical
>>> connections like a produce request going to the leader? Instead of one
>>> connection attempt to the leader, we want three separate connection
>>> attempts within the request timeout to the leader?
>>> 
>>> Regards,
>>> 
>>> Rajini
>>> 
>>> 
>>>> On Thu, May 7, 2020 at 11:51 PM Jose Garcia Sancio <
>> jsan...@confluent.io>
>>>> wrote:
>>>> Cheng,
>>>> Thanks for the KIP and the detailed proposal section. LGTM!
>>>>>> On Thu, May 7, 2020 at 3:38 PM Cheng Tan  wrote:
>>>>>> I think more about the potential wider use cases, I modified the
>>>> proposal to target all the connection. Thanks.
>>>>> - Best, - Cheng Tan
>>>>>> On May 7, 2020, at 1:41 AM, Cheng Tan  wrote:
>>>>>> Hi Colin,
>>>>>> Sorry for the confusion. 

Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-13 Thread Cheng Tan
Hi Rajini,

Thanks for the comments. 

> I think
> they started off as connection timeouts but now include authentication time
> as well. Have we considered using similar configs for this case?


The new config I proposed is focusing on the connections to unreachable 
servers. The timeout count won’t involved the authentication. I think “socket.” 
prefix make sense. I’ll change it. Colin mentioned that adding the key word 
“setup” can help people understand that this config only applies to the 
connection setup. What about “socket.connection.setup.ms”?

> The KIP proposes 10s as the default. What does this mean for typical
> connections like a produce request going to the leader?

The new timeout config applies to each connection. It’s at the NetworkClient 
level and won’t consider the underlying connection logic. Specifically, by 
default, every connection will have 10 seconds to become “connected” from 
“connecting”, which implies the corresponding socket channel is now connected 
(SocketChanel.finishConnect() returns True), no matter what the request logic 
and abstraction is.

Please let me know if these make sense to you or if you have more thoughts. 
Thank you.

Best, -Cheng

> 在 2020年5月13日,上午7:11,Rajini Sivaram  写道:
> 
> Hi Cheng,
> 
> Thanks for the KIP, sounds like a good improvement. A couple of comments:
> 
> 1) We currently have client connection timeouts on the broker with configs
> named `xxx.socket.timeout.ms` (e.g. controller.socket.timeout.ms). I think
> they started off as connection timeouts but now include authentication time
> as well. Have we considered using similar configs for this case? We may
> want to prefix the new config with `socket.` anyway - something along the
> lines of `socket.connection.timeout.ms` if it is just the connection time.
> 
> 2) The KIP proposes 10s as the default. What does this mean for typical
> connections like a produce request going to the leader? Instead of one
> connection attempt to the leader, we want three separate connection
> attempts within the request timeout to the leader?
> 
> Regards,
> 
> Rajini
> 
> 
>> On Thu, May 7, 2020 at 11:51 PM Jose Garcia Sancio 
>> wrote:
>> Cheng,
>> Thanks for the KIP and the detailed proposal section. LGTM!
>>>> On Thu, May 7, 2020 at 3:38 PM Cheng Tan  wrote:
>>>> I think more about the potential wider use cases, I modified the
>> proposal to target all the connection. Thanks.
>>> - Best, - Cheng Tan
>>>> On May 7, 2020, at 1:41 AM, Cheng Tan  wrote:
>>>> Hi Colin,
>>>> Sorry for the confusion. I’m proposing to implement timeout in the
>> NetworkClient.leastLoadedNode() when iterating all the cached node. The
>> alternative I can think is to implement the timeout in NetworkClient.poll()
>>>> I’d prefer to implement in the leastLoadedNode(). Here’re the reasons:
>>>> Usually when clients send a request, they will asking the network
>> client to send the request to a specific node. In this case, the
>> connection.setup.timeout won’t matter too much because the client doesn’t
>> want to try other nodes for that specific request. The request level
>> timeout would be enough. The metadata fetcher fetches the nodes status
>> periodically so the clients can reassign the request to another node after
>> timeout.
>>>> Consumer, producer, and AdminClient are all using leastLoadedNode()
>> for metadata fetch, where the connection setup timeout can play an
>> important role. Unlike other requests can refer to the metadata for node
>> condition, the metadata requests can only blindly choose a node for retry
>> in the worst scenario. We want to make sure the client can get the metadata
>> smoothly and as soon as possible. As a result, we need this
>> connection.setup.timeout.
>>>> Implementing the timeout in poll() or anywhere else might need an
>> extra iteration of all nodes, which might downgrade the network client
>> performance.
>>>> I also updated the KIP content and KIP status. Please let me know if
>> the above ideas make sense. Thanks.
>>>> Best, - Cheng Tan
>>>>> On May 4, 2020, at 5:26 PM, Colin McCabe > cmcc...@apache.org>> wrote:
>>>>> Hi Cheng,
>>>>> On the KIP page, it lists this KIP as "draft."  It seems like "under
>> discussion" is appropriate here, right?
>>>>>> Currently, the initial socket connection timeout is depending on
>> Linux kernel setting
>>>>>> tcp_syn_retries. The timeout value is 2 ^ (tcp_sync_retries + 1) - 1
>> seconds. For the
>>>>>> reasons below, we want to control the client-side sock

[jira] [Created] (KAFKA-9980) Reserved word "" may not be santinized to "%3Cdefault%3E"

2020-05-12 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9980:


 Summary: Reserved word "" may not be santinized to 
"%3Cdefault%3E"
 Key: KAFKA-9980
 URL: https://issues.apache.org/jira/browse/KAFKA-9980
 Project: Kafka
  Issue Type: Bug
        Reporter: Cheng Tan


quota_tests.py is failing. Specifically for this test:
[INFO:2020-05-11 19:22:47,493]: RunnerClient: Loading test \{'directory': 
'/opt/kafka-dev/tests/kafkatest/tests/client', 'file_name': 'quota_test.py', 
'method_name': 'test_quota', 'cls_name': 'QuotaTest', 'injected_args': 
{'quota_type': 'client-id', 'override_quota': False}}
I log into the docker container and do
/opt/kafka-dev/bin/kafka-configs.sh --bootstrap-server ducker03:9093 --describe 
--entity-type clients --command-config /opt/kafka-dev/bin/hi.properties
and the command return
Configs for the default client-id are consumer_byte_rate=200.0, 
producer_byte_rate=250.0
Configs for client-id 'overridden_id' are consumer_byte_rate=1.0E9, 
producer_byte_rate=1.0E9
Seems like the config is properly but the quota is not effective
 
For investigation, I added a logging at {{AdminZKClient.changeConfigs()}}
  def changeConfigs(entityType: String, entityName: String, configs: 
Properties): Unit = {
warn(s"entityType = $entityType entityName = $entityName configs = 
$configs")
...
  }
And use --bootstrap-server and --zookeeper to --alter the default client quota. 
I got
Alter with --zookeeper:WARN entityType = clients entityName =  configs 
= \{producer_byte_rate=10, consumer_byte_rate=10} 
(kafka.zk.AdminZkClient)
and
Alter with --bootstrap-server:WARN entityType = clients entityName = 
%3Cdefault%3E configs = \{producer_byte_rate=10, 
consumer_byte_rate=10} (kafka.zk.AdminZkClient)
I guess the encoding difference might cause the issue. The encoding happens in
Sanitizer.sanitize()



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [DISCUSS] KIP-554: Add Broker-side SCRAM Config API

2020-05-10 Thread Cheng Tan
Hi Colin,


If I understood correctly, in your design, listScramUsers will return the 
mechanism and iteration. Let’s use the field naming of RFC 5802 for this 
discussion:

 SaltedPassword  := Hi(Normalize(password), salt, i)
 ClientKey   := HMAC(SaltedPassword, "Client Key")
 StoredKey   := H(ClientKey)
 AuthMessage := client-first-message-bare + "," +
server-first-message + "," +
client-final-message-without-proof
 ClientSignature := HMAC(StoredKey, AuthMessage)
 ClientProof := ClientKey XOR ClientSignature
 ServerKey   := HMAC(SaltedPassword, "Server Key")
 ServerSignature := HMAC(ServerKey, AuthMessage)

I think it’s also safe and useful for listScramUsers to return salt and 
ServerKey. The current practice of —describe with —zookeeper is returning these 
two fields (KIP-84)

bin/kafka-configs.sh --zookeeper localhost:2181 --describe --entity-type users 
--entity-name alice
Configs for user-principal 'alice' are 
SCRAM-SHA-512=[salt=djR5dXdtZGNqamVpeml6NGhiZmMwY3hrbg==,stored_key=sb5jkqStV9RwPVTGxG1ZJHxF89bqjsD1jT4SFDK4An2goSnWpbNdY0nkq0fNV8xFcZqb7MVMJ1tyEgif5OXKDQ==,
 
server_key=3EfuHB4LPOcjDH0O5AysSSPiLskQfM5K9+mOzGmkixasmWEGJWZv7svtgkP+acO2Q9ms9WQQ9EndAJCvKHmjjg==,iterations=4096],SCRAM-SHA-256=[salt=10ibs0z7xzlu6w5ns0n188sis5,stored_key=+Acl/wi1vLZ95Uqj8rRHVcSp6qrdfQIwZbaZBwM0yvo=,server_key=nN+fZauE6vG0hmFAEj/49+2yk0803y67WSXMYkgh77k=,iterations=4096]


Please let me know what you think.

Best, - Cheng Tan

> On Apr 30, 2020, at 11:16 PM, Colin McCabe  wrote:
> 
> 



Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-07 Thread Cheng Tan
I think more about the potential wider use cases, I modified the proposal to 
target all the connection. Thanks.

- Best, - Cheng Tan

> On May 7, 2020, at 1:41 AM, Cheng Tan  wrote:
> 
> Hi Colin,
> 
> Sorry for the confusion. I’m proposing to implement timeout in the 
> NetworkClient.leastLoadedNode() when iterating all the cached node. The 
> alternative I can think is to implement the timeout in NetworkClient.poll() 
> 
> I’d prefer to implement in the leastLoadedNode(). Here’re the reasons:
> Usually when clients send a request, they will asking the network client to 
> send the request to a specific node. In this case, the 
> connection.setup.timeout won’t matter too much because the client doesn’t 
> want to try other nodes for that specific request. The request level timeout 
> would be enough. The metadata fetcher fetches the nodes status periodically 
> so the clients can reassign the request to another node after timeout.
> Consumer, producer, and AdminClient are all using leastLoadedNode() for 
> metadata fetch, where the connection setup timeout can play an important 
> role. Unlike other requests can refer to the metadata for node condition, the 
> metadata requests can only blindly choose a node for retry in the worst 
> scenario. We want to make sure the client can get the metadata smoothly and 
> as soon as possible. As a result, we need this connection.setup.timeout.
> Implementing the timeout in poll() or anywhere else might need an extra 
> iteration of all nodes, which might downgrade the network client performance.
> I also updated the KIP content and KIP status. Please let me know if the 
> above ideas make sense. Thanks.
> 
> Best, - Cheng Tan
> 
> 
> 
>> On May 4, 2020, at 5:26 PM, Colin McCabe > <mailto:cmcc...@apache.org>> wrote:
>> 
>> Hi Cheng,
>> 
>> On the KIP page, it lists this KIP as "draft."  It seems like "under 
>> discussion" is appropriate here, right?
>> 
>>> Currently, the initial socket connection timeout is depending on Linux 
>>> kernel setting
>>> tcp_syn_retries. The timeout value is 2 ^ (tcp_sync_retries + 1) - 1 
>>> seconds. For the
>>> reasons below, we want to control the client-side socket timeout directly 
>>> using 
>>> configuration files
>> 
>> Linux is just one example of an OS that Kafka could run on, right?  You 
>> could also be running on MacOS, for example.
>> 
>>> I'm proposing to do a lazy socket connection time out. That is, we only 
>>> check if
>>> we need to timeout a socket when we consider the corresponding node as a 
>>> candidate in the node provider.
>> 
>> The NodeProvider is an AdminClient abstraction, right?  Why wouldn't we 
>> implement a connection setup timeout for all clients, not just AdminClient?
>> 
>> best,
>> Colin
>> 
>> On Mon, May 4, 2020, at 13:18, Colin McCabe wrote:
>>> Hmm.  A big part of the reason behind the KIP is that the default 
>>> connection timeout behavior of the OS doesn't work for Kafka, right?  
>>> For example, on Linux, if we wait 127 seconds for a connection attempt 
>>> to time out, we won't get a chance to make another attempt in most 
>>> cases.  So I think it makes sense to set a shorter default.
>>> 
>>> best,
>>> Colin
>>> 
>>> 
>>> On Mon, May 4, 2020, at 09:44, Jose Garcia Sancio wrote:
>>>> Thanks for the KIP Cheng,
>>>> 
>>>>> The default value will be 10 seconds.
>>>> 
>>>> I think we should make the default the current behavior. Meaning the
>>>> default should leverage the default connect timeout from the operating
>>>> system.
>>>> 
>>>>> Proposed Changes
>>>> 
>>>> I don't fully understand this section. It seems like it is mainly
>>>> focused on the problem with the current implementation. Can you
>>>> explain how the proposed changes solve the problem?
>>>> 
>>>> Thanks.
>>>> 
>>>> 
>>>> -- 
>>>> -Jose
>>>> 
>>> 
> 



Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-07 Thread Cheng Tan
Hi Colin,

Sorry for the confusion. I’m proposing to implement timeout in the 
NetworkClient.leastLoadedNode() when iterating all the cached node. The 
alternative I can think is to implement the timeout in NetworkClient.poll() 

I’d prefer to implement in the leastLoadedNode(). Here’re the reasons:
Usually when clients send a request, they will asking the network client to 
send the request to a specific node. In this case, the connection.setup.timeout 
won’t matter too much because the client doesn’t want to try other nodes for 
that specific request. The request level timeout would be enough. The metadata 
fetcher fetches the nodes status periodically so the clients can reassign the 
request to another node after timeout.
Consumer, producer, and AdminClient are all using leastLoadedNode() for 
metadata fetch, where the connection setup timeout can play an important role. 
Unlike other requests can refer to the metadata for node condition, the 
metadata requests can only blindly choose a node for retry in the worst 
scenario. We want to make sure the client can get the metadata smoothly and as 
soon as possible. As a result, we need this connection.setup.timeout.
Implementing the timeout in poll() or anywhere else might need an extra 
iteration of all nodes, which might downgrade the network client performance.
I also updated the KIP content and KIP status. Please let me know if the above 
ideas make sense. Thanks.

Best, - Cheng Tan



> On May 4, 2020, at 5:26 PM, Colin McCabe  wrote:
> 
> Hi Cheng,
> 
> On the KIP page, it lists this KIP as "draft."  It seems like "under 
> discussion" is appropriate here, right?
> 
>> Currently, the initial socket connection timeout is depending on Linux 
>> kernel setting
>> tcp_syn_retries. The timeout value is 2 ^ (tcp_sync_retries + 1) - 1 
>> seconds. For the
>> reasons below, we want to control the client-side socket timeout directly 
>> using 
>> configuration files
> 
> Linux is just one example of an OS that Kafka could run on, right?  You could 
> also be running on MacOS, for example.
> 
>> I'm proposing to do a lazy socket connection time out. That is, we only 
>> check if
>> we need to timeout a socket when we consider the corresponding node as a 
>> candidate in the node provider.
> 
> The NodeProvider is an AdminClient abstraction, right?  Why wouldn't we 
> implement a connection setup timeout for all clients, not just AdminClient?
> 
> best,
> Colin
> 
> On Mon, May 4, 2020, at 13:18, Colin McCabe wrote:
>> Hmm.  A big part of the reason behind the KIP is that the default 
>> connection timeout behavior of the OS doesn't work for Kafka, right?  
>> For example, on Linux, if we wait 127 seconds for a connection attempt 
>> to time out, we won't get a chance to make another attempt in most 
>> cases.  So I think it makes sense to set a shorter default.
>> 
>> best,
>> Colin
>> 
>> 
>> On Mon, May 4, 2020, at 09:44, Jose Garcia Sancio wrote:
>>> Thanks for the KIP Cheng,
>>> 
>>>> The default value will be 10 seconds.
>>> 
>>> I think we should make the default the current behavior. Meaning the
>>> default should leverage the default connect timeout from the operating
>>> system.
>>> 
>>>> Proposed Changes
>>> 
>>> I don't fully understand this section. It seems like it is mainly
>>> focused on the problem with the current implementation. Can you
>>> explain how the proposed changes solve the problem?
>>> 
>>> Thanks.
>>> 
>>> 
>>> -- 
>>> -Jose
>>> 
>> 



[jira] [Created] (KAFKA-9959) leastLoadedNode() does not provide a node fairly

2020-05-05 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9959:


 Summary: leastLoadedNode() does not provide a node fairly
 Key: KAFKA-9959
 URL: https://issues.apache.org/jira/browse/KAFKA-9959
 Project: Kafka
  Issue Type: Bug
Reporter: Cheng Tan






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-9954) Config command didn't validate the unsupported user config change

2020-05-04 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9954:


 Summary: Config command didn't validate the unsupported user 
config change
 Key: KAFKA-9954
 URL: https://issues.apache.org/jira/browse/KAFKA-9954
 Project: Kafka
  Issue Type: Bug
Reporter: Cheng Tan


{quote}bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter 
--add-config producer_byte_rate=4 --entity-type users --entity-default
{quote}
 

will say that the alternation is complete. However, we don't support the 
alternation yet.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [DISCUSS] KIP-601: Configurable socket connection timeout

2020-05-01 Thread Cheng Tan
Hi Colin. Thanks for the discussion and feedback. I re-wrote the KIP-601 
proposal following your suggestions. Now the new proposal is ready.

Best, - Cheng Tan

> On Apr 28, 2020, at 2:55 PM, Colin McCabe  wrote:
> 
> 
> Thanks again for the KIP.  This seems like it has been a gap in Kaf



[jira] [Created] (KAFKA-9942) --entity-default flag is not working for alternating configs in AdminClient

2020-04-29 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9942:


 Summary: --entity-default flag is not working for alternating 
configs in AdminClient
 Key: KAFKA-9942
 URL: https://issues.apache.org/jira/browse/KAFKA-9942
 Project: Kafka
  Issue Type: Bug
Reporter: Cheng Tan


$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 
producer_byte_rate=10,consumer_byte_rate=10 --entity-type 
clients --entity-default

The command above will trigger the exception below

java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.UnknownServerException: Path must not end with / 
character

 at 
org.apache.kafka.common.internals.KafkaFutureImpl.wrapAndThrow(KafkaFutureImpl.java:45)

 at 
org.apache.kafka.common.internals.KafkaFutureImpl.access$000(KafkaFutureImpl.java:32)

 at 
org.apache.kafka.common.internals.KafkaFutureImpl$SingleWaiter.await(KafkaFutureImpl.java:104)

 at 
org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:272)

 at 
kafka.admin.ConfigCommand$.getAllClientQuotasConfigs(ConfigCommand.scala:501)

 at kafka.admin.ConfigCommand$.getClientQuotasConfig(ConfigCommand.scala:487)

 at kafka.admin.ConfigCommand$.alterConfig(ConfigCommand.scala:361)

 at kafka.admin.ConfigCommand$.processCommand(ConfigCommand.scala:292)

 at kafka.admin.ConfigCommand$.main(ConfigCommand.scala:91)

 at kafka.admin.ConfigCommand.main(ConfigCommand.scala)

Caused by: org.apache.kafka.common.errors.UnknownServerException: Path must not 
end with / character



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[DISCUSS] KIP-601: Configurable socket connection timeout

2020-04-27 Thread Cheng Tan
Hi developers,


I’m proposing KIP-601 to support configurable socket connection timeout. 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-601%3A+Configurable+socket+connection+timeout
 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-601:+Configurable+socket+connection+timeout>

Currently, the initial socket connection timeout is depending on system setting 
tcp_syn_retries. The actual timeout value is 2 ^ (tcp_sync_retries + 1) - 1 
seconds. For the reasons below, we want to control the client-side socket 
timeout directly using configuration files. 
• The default value of tcp_syn_retries is 6. It means the default 
timeout value is 127 seconds and too long in some scenarios. For example, when 
the user specifies a list of N bootstrap-servers and no connection has been 
built between the client and the servers, the least loaded node provider will 
poll all the server nodes specified by the user. If M servers in the 
bootstrap-servers list are offline, the client may take (127 * M) seconds to 
connect to the cluster. In the worst case when M = N - 1, the wait time can be 
several minutes.
• Though we may set the default value of tcp_syn_retries smaller, we 
will then change the system level network behaviors, which might cause other 
issues.
• Applications depending on KafkaAdminClient may want to robustly know 
and control the initial socket connect timeout, which can help throw 
corresponding exceptions in their layer.

Please let me know if you have any thoughts or suggestions. Thanks.


Best, - Cheng Tan



[jira] [Created] (KAFKA-9893) Configurable TCP connection timeout for AdminClient

2020-04-20 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9893:


 Summary: Configurable TCP connection timeout for AdminClient
 Key: KAFKA-9893
 URL: https://issues.apache.org/jira/browse/KAFKA-9893
 Project: Kafka
  Issue Type: New Feature
Reporter: Cheng Tan


We do not currently allow for connection timeouts to be defined within 
AdminClient, and as a result rely on the default OS settings to determine 
whether a broker is inactive before selecting an alternate broker from 
bootstrap.

In the case of a connection timeout on initial handshake, and where 
tcp_syn_retries is the default (6), we won't timeout an unresponsive broker 
until ~127s - while the client will timeout sooner (~120s).

Reducing tcp_syn_retries should mitigate the issue depending on the number of 
unresponsive brokers within the bootstrap, though this will be applied system 
wide, and it would be good if we could instead configure connection timeouts 
for AdminClient.

The use case where this came up was a customer performing DC failover tests 
with a stretch cluster.

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[DISCUSS] KIP-593: Enable --if-exists and --if-not-exists for AdminClient in TopicCommand

2020-04-13 Thread Cheng Tan
Hi developers,

In kafka-topic.sh, we expect to use --if-exists to ensure that the topic to 
create or change exists. Similarly, we expect to use --if-not-exists to ensure 
that the topic to create or change does not exist. Currently, only 
ZookeeperTopicService supports these two options. We want to introduce them to 
AdminClientTopicService. Please let me know if you have any thought or idea 
related to this KIP. 

Here’s the link to the KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-593%3A+Enable+--if-exists+and+--if-not-exists+for+AdminClient+in+TopicCommand
 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-593:+Enable+--if-exists+and+--if-not-exists+for+AdminClient+in+TopicCommand>

Thanks,

- Cheng Tan

[jira] [Created] (KAFKA-9862) Enable --if-exists and --if-not-exists for AdminClient in TopicCommand

2020-04-13 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9862:


 Summary: Enable --if-exists and --if-not-exists for AdminClient in 
TopicCommand
 Key: KAFKA-9862
 URL: https://issues.apache.org/jira/browse/KAFKA-9862
 Project: Kafka
  Issue Type: New Feature
Reporter: Cheng Tan
Assignee: Cheng Tan


In *kafka-topic.sh*, we expect to use --if-exists to ensure that the topic to 
create or change exists. Similarly, we expect to use --if-not-exists to ensure 
that the topic to create or change does not exist. Currently, only 
*ZookeeperTopicService* supports these two options and we want to introduce 
them to *AdminClientTopicService.*



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (KAFKA-9800) [KIP-580] Admin Client Exponential Backoff Implementation

2020-04-01 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9800:


 Summary: [KIP-580] Admin Client Exponential Backoff Implementation
 Key: KAFKA-9800
 URL: https://issues.apache.org/jira/browse/KAFKA-9800
 Project: Kafka
  Issue Type: New Feature
Reporter: Cheng Tan
Assignee: Cheng Tan


In {{KafkaAdminClient}}, we will have to modify the way the retry backoff is 
calculated for the calls that have failed and need to be retried. From the 
current static retry backoff, we have to introduce a mechanism for all calls 
that upon failure, the next retry time is dynamically calculated.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[DISCUSS] KIP-576 Support dynamic update of more broker configs related to replication

2020-03-23 Thread Cheng Tan
Hi Everyone,

KIP-226 added support for dynamic update of  broker configuration.  In this 
KIP, we propose to extend the support to dynamic update of a group of new 
dynamic broker configs which will benefit the replication process.

The proposed support for new dynamic configs looks like

failed.authentication.delay Connection close delay on failed authentication.
fetch.max.bytes Maximum bytes expected for the entire fetch response.
replica.fetch.backoff.msThe amount of time to sleep when fetch 
partition error occurs.
replica.fetch.response.max.bytesMaximum bytes expected for the entire 
fetch response.
replica.fetch.response.max.bytesMaximum bytes expected for the entire 
fetch response. 
replica.fetch.wait.max.ms   Max wait time for each fetcher request issued 
by follower replicas.


I’ve written a KIP about introducing exponential backoff for Kafka clients. 
Would appreciate any feedback on this. Thanks.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-580%3A+Exponential+Backoff+for+Kafka+Clients
 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-580:+Exponential+Backoff+for+Kafka+Clients>

Best, - Cheng Tan



Re: [DISCUSS] KIP-580: Exponential Backoff for Kafka Clients

2020-03-23 Thread Cheng Tan
+1 (non-binding)

> On Mar 19, 2020, at 7:27 PM, Sanjana Kaundinya  wrote:
> 
> Ah yes that makes sense. I’ll update the KIP to reflect this.
> 
> Thanks,
> Sanjana
> 
> On Thu, Mar 19, 2020 at 5:48 PM Guozhang Wang  wrote:
> 
>> Following the formula you have in the KIP, if it is simply:
>> 
>> MIN(retry.backoff.max.ms, (retry.backoff.ms * 2**(failures - 1)) * random(
>> 0.8, 1.2))
>> 
>> then the behavior would stay consistent at retry.backoff.max.ms.
>> 
>> 
>> Guozhang
>> 
>> On Thu, Mar 19, 2020 at 5:46 PM Sanjana Kaundinya 
>> wrote:
>> 
>>> If that’s the case then what should we base the starting point as?
>>> Currently in the KIP the starting point is retry.backoff.ms and it
>>> exponentially goes up to retry.backoff.max.ms. If retry.backoff.max.ms
>> is
>>> smaller than retry.backoff.ms then that could pose a bit of a problem
>>> there right?
>>> 
>>> On Mar 19, 2020, 5:44 PM -0700, Guozhang Wang ,
>> wrote:
 Thanks Sanjana, I did not capture the part that Jason referred to, so
 that's my bad :P
 
 Regarding your last statement, I actually feel that instead of take the
 larger of the two, we should respect "retry.backoff.max.ms" even if it
>>> is
 smaller than "retry.backoff.ms". I do not have a very strong rationale
 except it is logically more aligned to the config names.
 
 
 Guozhang
 
 
 On Thu, Mar 19, 2020 at 5:39 PM Sanjana Kaundinya <
>> skaundi...@gmail.com>
 wrote:
 
> Hey Jason and Guozhang,
> 
> Jason is right, I took this inspiration from KIP-144 (
> 
> 
>>> 
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-144%3A+Exponential+backoff+for+broker+reconnect+attempts
> )
> which had the same logic in order to preserve the existing behavior.
>> In
> this case however, if we are thinking to completely eliminate the
>>> static
> backoff behavior, we can do that and as Jason mentioned put it in the
> release notes and not add any special logic. In addition I agree that
>>> we
> should take the larger of the two of `retry.backoff.ms` and `
> retry.backoff.max.ms`. I'll update the KIP to reflect this and make
>> it
> clear that the old static retry backoff is getting replaced by the
>> new
> dynamic retry backoff.
> 
> Thanks,
> Sanjana
> On Thu, Mar 19, 2020 at 4:23 PM Jason Gustafson 
> wrote:
> 
>> Hey Guozhang,
>> 
>> I was referring to this:
>> 
>>> For users who have not set retry.backoff.ms explicitly, the
>>> default
>> behavior will change so that the backoff will grow up to 1000 ms.
>> For
> users
>> who have set retry.backoff.ms explicitly, the behavior will remain
>>> the
>> same
>> as they could have specific requirements.
>> 
>> I took this to mean that for users who have overridden `
>>> retry.backoff.ms
> `
>> to 50ms (say), we will change the default `retry.backoff.max.ms`
>> to
>>> 50ms
>> as
>> well in order to preserve existing backoff behavior. Is that not
>>> right?
> In
>> any case, I agree that we can use the maximum of the two values as
>>> the
>> effective `retry.backoff.max.ms` to handle the case when the
>>> configured
>> value of `retry.backoff.ms` is larger than the default of 1s.
>> 
>> -Jason
>> 
>> 
>> 
>> 
>> On Thu, Mar 19, 2020 at 3:29 PM Guozhang Wang 
> wrote:
>> 
>>> Hey Jason,
>>> 
>>> My understanding is a bit different here: even if user has an
>>> explicit
>>> overridden "retry.backoff.ms", the exponential mechanism still
> triggers
>>> and
>>> the backoff would be increased till "retry.backoff.max.ms"; and
>>> if the
>>> specified "retry.backoff.ms" is already larger than the "
>>> retry.backoff.max.ms", we would still take "retry.backoff.max.ms
>> ".
>>> 
>>> So if the user does override the "retry.backoff.ms" to a value
>>> larger
>> than
>>> 1s and is not aware of the new config, she would be surprised to
>>> see
> the
>>> specified value seemingly not being respected, but she could
>> still
> learn
>>> that afterwards by reading the release notes introducing this KIP
>> anyways.
>>> 
>>> 
>>> Guozhang
>>> 
>>> On Thu, Mar 19, 2020 at 3:10 PM Jason Gustafson <
>>> ja...@confluent.io>
>>> wrote:
>>> 
 Hi Sanjana,
 
 The KIP looks good to me. I had just one question about the
>>> default
 behavior. As I understand, if the user has specified `
> retry.backoff.ms
>> `
 explicitly, then we will not apply the default max backoff. As
>>> such,
 there's no way to get the benefit of this feature if you are
> providing
>> a
>>> `
 retry.backoff.ms` unless you also provide `
>> retry.backoff.max.ms
>>> `.
> That
 makes sense if you assume the user is unaware of the new
> configuration,
>>> but
 it i

[DISCUSS] KIP-576

2020-03-08 Thread Cheng Tan
Hi developers,

KIP-226 added support for dynamic update of  broker configuration.  In this 
KIP, we propose to extend the support to dynamic update of a group of new 
dynamic broker configs which will benefit the replication process. In this KIP, 
we propose to extend the support to dynamic update of a group of new dynamic 
broker configs which will benefit the replication process.

fetch.max.bytes Maximum bytes expected for the entire fetch response.
failed.authentication.delay Connection close delay on failed authentication.
replica.fetch.response.max.bytesMaximum bytes expected for the entire 
fetch response.
replica.fetch.wait.max.ms   Max wait time for each fetcher request issued 
by follower replicas.
follower.replication.throttled.replicas A list of replicas for which log 
replication should be throttled on the follower side.
leader.replication.throttled.replicas   A list of replicas for which log 
replication should be throttled on the leader side.


The wiki link for KIP-576:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-576%3A+Support+dynamic+update+of+more+configs+related+to+replication
 
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-576:+Support+dynamic+update+of+more+configs+related+to+replication>

Please feel free to discuss and suggest any other broker configs worth making 
dynamic. Thanks.

Regards, - Cheng Tan

[jira] [Created] (KAFKA-9683) Support dynamic update of fetch.max.bytes/failed.authentication.delay/replica.fetch.response.max.bytes/replica.fetch.wait.max.ms/follower.replication.throttled.replicas/l

2020-03-08 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9683:


 Summary: Support dynamic update of 
fetch.max.bytes/failed.authentication.delay/replica.fetch.response.max.bytes/replica.fetch.wait.max.ms/follower.replication.throttled.replicas/leader.replication.throttled.replicas
 Key: KAFKA-9683
 URL: https://issues.apache.org/jira/browse/KAFKA-9683
 Project: Kafka
  Issue Type: Bug
Reporter: Cheng Tan


[KIP-226|https://cwiki.apache.org/confluence/display/KAFKA/KIP-226+-+Dynamic+Broker+Configuration]
 added support for dynamic update of  broker configuration.  In this KIP, we 
propose to extend the support to dynamic update of a group of new dynamic 
broker configs which will benefit the replication process.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Create content permission

2020-03-06 Thread Cheng Tan
It’s d8tltanc. Thank you,

> On Mar 6, 2020, at 9:35 AM, Matthias J. Sax  wrote:
> 
> -BEGIN PGP SIGNED MESSAGE-
> Hash: SHA512
> 
> What is your wiki account id?
> 
> - -Matthias
> 
> On 3/5/20 2:15 PM, Cheng Tan wrote:
>> Hello,
>> 
>> My name is Cheng Tan and I’m working at Confluent. Can I get the
>> create content permission so I can create a KIP? Thank you.
>> 
>> Best, - Cheng Tan
>> 
> -BEGIN PGP SIGNATURE-
> 
> iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl5iiecACgkQO4miYXKq
> /OjBww//U6258HTqWZZ7DlWNTYmrjw+ksQE60BqeMhaNWI3gsMhiuzoihprr3iUP
> sxiCwdeGGnDmP45whUDhrwzGtH0p9NaRct2E6+s6CkrwTZX0jut/HH2ytM7tqYVh
> JJ8VvYNoBt1BI7gWetoaFHtIeRfbo0Y+k8KtkUHw630o+3gIMuPjs8lZvUgsTiy7
> hgjaHXItSbHpvc72PtFpXHc4XKBqhp9goO6DHQ8LCjoJPvN+62QQwjC6E8fjDhcH
> 2wI3Je16Yuo74SDuCSxEDpfoAmzIR74916KrbhYTjAUlYDZWkmAjWvzLOLjisK2B
> Bsf0F7ihpigHTT07bQitENZS0mQ+jyU6UaHpsqAnGyaEUFr4UXgqK8z87ByTgp9C
> zhTBjELgnk6USCFkPvptgJx6kBqXb9lj5PkBhMEGJupoZC9LaQMSNQqXUtU0rPPm
> sf35tlWaMiY/SVE3pnb/6Km7M+z5+oGRL9nxp97JRxxHwBThf3+CLhzd2+3HJozp
> xCbnQKIINxgOJT0Ckj4rsMe2vYaZ79y1m054PtP2PteF24yaJTP46esXz27lba4T
> O0e0FoIeiYYGat2ReBpOb10+qxx2EICzL6fvQcaZSLDbN3HJ7a3aB1ktjLh6ZqW6
> OCxLZBIwAzCDPNAlmZLzYZJ9CcgAbWLiKz5FgEQdkZk7I1koIg0=
> =LONO
> -END PGP SIGNATURE-



Create content permission

2020-03-05 Thread Cheng Tan
Hello,

My name is Cheng Tan and I’m working at Confluent. Can I get the create content 
permission so I can create a KIP? Thank you.

Best, - Cheng Tan

[jira] [Created] (KAFKA-9591) Log clearer error messages when there is an offset out of range (Client Change)

2020-02-21 Thread Cheng Tan (Jira)
Cheng Tan created KAFKA-9591:


 Summary: Log clearer error messages when there is an offset out of 
range (Client Change)
 Key: KAFKA-9591
 URL: https://issues.apache.org/jira/browse/KAFKA-9591
 Project: Kafka
  Issue Type: Bug
Reporter: Cheng Tan
Assignee: Cheng Tan


Client change related to KAFKA-9580



--
This message was sent by Atlassian Jira
(v8.3.4#803005)