Re: KRaft Observer Nodes

2024-03-25 Thread Paolo Patierno
indicate that we want a broker to be an observer? > > On Sun, Mar 24, 2024 at 3:01 PM Paolo Patierno > wrote: > > > Hi Sanaa, > > in KRaft mode there is the role of "observer" which is typically taken by > > brokers as part of the data plane. The broker/obs

Re: KRaft Observer Nodes

2024-03-24 Thread Paolo Patierno
Hi Sanaa, in KRaft mode there is the role of "observer" which is typically taken by brokers as part of the data plane. The broker/observer can discover which node is the leader/active controller and fetches the metadata from it (as the "followers" in the KRaft quorum), but cannot vote and cannot be

Re: [VOTE] 3.7.0 RC4

2024-02-14 Thread Paolo Patierno
+1 (non-binding). I used the staged binaries with Scala 2.13 and mostly focused on the ZooKeeper to KRaft migration with multiple tests. Everything works fine. Thanks Paolo On Mon, 12 Feb 2024, 22:06 Jakub Scholz, wrote: > +1 (non-binding). I used the staged binaries with Scala 2.13 and the sta

Re: [VOTE] 3.7.0 RC2

2024-01-15 Thread Paolo Patierno
ighly-experimental > >>> > > > > > single laptop run! > >>> > > > > > > >>> > > > > > > >>> > > > > > Cheers, > >>> > > > > > > >>> > > > > > Chris >

[jira] [Created] (KAFKA-16101) Kafka cluster unavailable during KRaft migration rollback procedure

2024-01-09 Thread Paolo Patierno (Jira)
Paolo Patierno created KAFKA-16101: -- Summary: Kafka cluster unavailable during KRaft migration rollback procedure Key: KAFKA-16101 URL: https://issues.apache.org/jira/browse/KAFKA-16101 Project

[jira] [Created] (KAFKA-16005) ZooKeeper to KRaft migration rollback missing disabling controller and migration configuration on brokers

2023-12-13 Thread Paolo Patierno (Jira)
Paolo Patierno created KAFKA-16005: -- Summary: ZooKeeper to KRaft migration rollback missing disabling controller and migration configuration on brokers Key: KAFKA-16005 URL: https://issues.apache.org/jira/browse

[jira] [Created] (KAFKA-16003) The znode /config/topics is not updated during KRaft migration in "dual-write" mode

2023-12-13 Thread Paolo Patierno (Jira)
Paolo Patierno created KAFKA-16003: -- Summary: The znode /config/topics is not updated during KRaft migration in "dual-write" mode Key: KAFKA-16003 URL: https://issues.apache.org/jira/browse/K

[jira] [Resolved] (KAFKA-15754) The kafka-storage tool can generate UUID starting with "-"

2023-10-31 Thread Paolo Patierno (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paolo Patierno resolved KAFKA-15754. Resolution: Not A Problem > The kafka-storage tool can generate UUID starting w

[jira] [Reopened] (KAFKA-15754) The kafka-storage tool can generate UUID starting with "-"

2023-10-31 Thread Paolo Patierno (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-15754?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paolo Patierno reopened KAFKA-15754: > The kafka-storage tool can generate UUID starting w

[jira] [Created] (KAFKA-15754) The kafka-storage tool can generate UUID starting with "-"

2023-10-30 Thread Paolo Patierno (Jira)
Paolo Patierno created KAFKA-15754: -- Summary: The kafka-storage tool can generate UUID starting with "-" Key: KAFKA-15754 URL: https://issues.apache.org/jira/browse/KAFKA-15754 Proj

[jira] [Created] (KAFKA-15689) KRaftMigrationDriver not logging the skipped event when expected state is wrong

2023-10-26 Thread Paolo Patierno (Jira)
Paolo Patierno created KAFKA-15689: -- Summary: KRaftMigrationDriver not logging the skipped event when expected state is wrong Key: KAFKA-15689 URL: https://issues.apache.org/jira/browse/KAFKA-15689

[jira] [Created] (KAFKA-14883) Broker state should be "observer" in KRaft quorum

2023-04-07 Thread Paolo Patierno (Jira)
Paolo Patierno created KAFKA-14883: -- Summary: Broker state should be "observer" in KRaft quorum Key: KAFKA-14883 URL: https://issues.apache.org/jira/browse/KAFKA-14883 Project: Kafka

[jira] [Created] (KAFKA-14411) Logging warning when partitions don't exist on assign request

2022-11-21 Thread Paolo Patierno (Jira)
Paolo Patierno created KAFKA-14411: -- Summary: Logging warning when partitions don't exist on assign request Key: KAFKA-14411 URL: https://issues.apache.org/jira/browse/KAFKA-14411 Project:

Re: [ANNOUNCE] New committer: Tom Bentley

2021-03-15 Thread Paolo Patierno
Congratulations Tom! Get Outlook for Android From: Guozhang Wang Sent: Monday, March 15, 2021 8:02:44 PM To: dev Subject: Re: [ANNOUNCE] New committer: Tom Bentley Congratulations Tom! Guozhang On Mon, Mar 15, 2021 at 11:25 AM Bill Beje

Re: [ANNOUNCE] New Kafka PMC member: Dong Lin

2018-08-20 Thread Paolo Patierno
Congratulations Dong! Paolo Patierno Principal Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno> Blog : De

KIP-176 and related PR : today deadline for merging in 2.0.0

2018-06-05 Thread Paolo Patierno
/confluence/display/KAFKA/KIP-176%3A+Remove+deprecated+new-consumer+option+for+tools Thanks Paolo Patierno Principal Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it

Re: [VOTE] KIP-176: Remove deprecated new-consumer option for tools

2018-05-29 Thread Paolo Patierno
Thanks for voting! As far as I understood, today is the last day for having a PR for this one. The KIP-176 has 3 (bindings) + 2 (non binding) votes so it is accepted. I'll provide a related PR by the EOD. Thanks, Paolo Patierno Principal Software Engineer (IoT) @ Red Hat Microsoft MVP on

[VOTE] KIP-176: Remove deprecated new-consumer option for tools

2018-05-23 Thread Paolo Patierno
n you vote for that adding to the release plan, please ? Thanks, Paolo Patierno Principal Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/p

Re: [ANNOUNCE] New Committer: Dong Lin

2018-03-28 Thread Paolo Patierno
Congrats Dong !! From: Becket Qin Sent: Wednesday, March 28, 2018 7:58:07 PM To: dev; us...@kafka.apache.org Subject: [ANNOUNCE] New Committer: Dong Lin Hello everyone, The PMC of Apache Kafka is pleased to announce that Dong Lin has accepted our invitation to be

Re: 1.1 release progress

2018-02-01 Thread Paolo Patierno
Hi Guozhang and Damian, I have a couple of issues marked for 1.1.0 but the PR isn't reviewed since a long time and both of them should be ready. They are : KAFKA-5919 KAFKA-5532 Maybe you can ta

Re: [DISCUSS] KIP-251: Allow timestamp manipulation in Processor API

2018-02-01 Thread Paolo Patierno
Hi Matthias, just a question : what will be the timestamp "type" in the new message on the wire ? Thanks, Paolo. From: Matthias J. Sax Sent: Wednesday, January 31, 2018 2:06 AM To: dev@kafka.apache.org Subject: [DISCUSS] KIP-251: Allow timestamp manipulation in

Re: [ANNOUNCE] New Kafka PMC Member: Rajini Sivaram

2018-01-17 Thread Paolo Patierno
Congratulations Rajini ! From: Sriram Subramanian Sent: Wednesday, January 17, 2018 8:01 PM To: dev@kafka.apache.org Subject: Re: [ANNOUNCE] New Kafka PMC Member: Rajini Sivaram Congratulations Rajini! On Wed, Jan 17, 2018 at 11:40 AM, Edoardo Comar wrote: > C

Re: [ANNOUNCE] New committer: Matthias J. Sax

2018-01-12 Thread Paolo Patierno
Congratulations Matthias ! Very well deserved ! From: Guozhang Wang Sent: Friday, January 12, 2018 11:59:21 PM To: dev@kafka.apache.org; us...@kafka.apache.org Subject: [ANNOUNCE] New committer: Matthias J. Sax Hello everyone, The PMC of Apache Kafka is pleased t

Re: Stream Processing Meetup@LinkedIn (Dec 4th)

2017-11-17 Thread Paolo Patierno
Hi Becket, I watched some of these meetups on the related YouTube channel in the past. Will be it available in streaming or just recorded for watching it later ? Thanks Paolo From: Becket Qin Sent: Friday, November 17, 2017 8:33:04 PM To: dev@kafka.apache.org; us.

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-15 Thread Paolo Patierno
Ismael +1 ... I'm going to update the name Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno> Blog

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-13 Thread Paolo Patierno
Guozhang thanks :-) It's getting late in my timezone, so maybe it's better for me don't take a look at email anymore ;) So finally, the KIP-204 was accepted. Waiting for more comments (if needed) on the PR for getting it merged. Thanks, Paolo Patierno Senior Software Engin

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-13 Thread Paolo Patierno
PM To: Paolo Patierno Cc: dev@kafka.apache.org Subject: Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API I'm not sure if Ismael's reply on the mailing list is a casted vote for this KIP. @Ismael, could you review the KIP again and cast your vote i

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-13 Thread Paolo Patierno
Hi all, I'm going to close this vote because this KIP was accepted with : 3 binding votes 5 non-binding votes Thanks everyone for comments and for voting. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppati

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-11 Thread Paolo Patierno
afka/pull/4132> as well. Thanks for your comments ! Paolo. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.c

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-10 Thread Paolo Patierno
the others opinions otherwise I'll start to implement in such way. I have updated the KIP and the PR using "recordsToDelete" parameter as well. Thanks Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter :

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-10 Thread Paolo Patierno
stead of having to document it via a comment only. Ismael On Tue, Oct 3, 2017 at 6:51 AM, Paolo Patierno wrote: > Hi all, > > I didn't see any further discussion around this KIP, so I'd like to start > the vote for it. > > Just for reference : https://cwiki.apache.org

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-07 Thread Paolo Patierno
the KIP got accepted and we can start to think about merging the PR for a future 1.1.0 release. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopat

Re: [ANNOUNCE] New committer: Onur Karaman

2017-11-07 Thread Paolo Patierno
Congrats Onur ! Well deserved ! Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno> Blog : De

Re: [ANNOUNCE] Apache Kafka 1.0.0 Released

2017-11-02 Thread Paolo Patierno
Congratulations for this milestone ! Thanks to Gouzhang for running the release ! Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.li

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-02 Thread Paolo Patierno
Hi Colin, I have just updated the KIP mentioning that this new method should replace the "legacy" Scala API used for deleting records today. Thanks, Paolo. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitte

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-02 Thread Paolo Patierno
Thanks Jason ! I have just updated the KIP with DeleteRecordsOptions definition. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.li

Re: Metadata class doesn't "expose" topics with errors

2017-10-31 Thread Paolo Patierno
@kafka.apache.org Subject: Re: Metadata class doesn't "expose" topics with errors Hello Paolo, I'm looking at your PR for KIP-204 now. Will reply on the discussion thread / PR diff file directly if I find anything. Guozhang On Tue, Oct 24, 2017 at 5:45 AM, Paolo Patierno wrote: > Hi G

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-31 Thread Paolo Patierno
t 25, 2017, at 03:47, Paolo Patierno wrote: > Thanks for all your feedback guys. I have updated my current code as > well. > > I know that the vote for this KIP is not started yet (actually I opened > it due to no feedback on this KIP after a while but then the discussion > started

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-31 Thread Paolo Patierno
tober 18 ? There are only "non binding" votes up to now. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedi

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-25 Thread Paolo Patierno
. Maybe feedback could be useful on that as well : https://github.com/apache/kafka/pull/4132 Thanks Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopat

Re: Metadata class doesn't "expose" topics with errors

2017-10-24 Thread Paolo Patierno
;legacy" admin client). For now the current implementation I have (I'll push a PR soon), use the Call class for sending a MetadataRequest and then its handleResponse for using another Call instance for sending the DeleteRecordsRequest. Thanks Paolo Patierno Senior Software Engineer (IoT)

Re: Metadata class doesn't "expose" topics with errors

2017-10-23 Thread Paolo Patierno
Finally another plan could be to use nesting of runnable calls. The first one for asking metadata (using the MetadataRequest which provides us all the errors) and then sending the delete records requests in the handleResponse() of such metadata request. Paolo Patierno Senior Software Engineer

Re: Metadata class doesn't "expose" topics with errors

2017-10-23 Thread Paolo Patierno
y one requests but different of them (even if for the same leader). Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/p

Metadata class doesn't "expose" topics with errors

2017-10-23 Thread Paolo Patierno
ors" are not exposed in the Metadata instance ? Is the preferred pattern using the low level protocol stuff in such case ? Thanks Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.c

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-23 Thread Paolo Patierno
ted to the delete records operation and what it means, so the target for such operation. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<ht

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-23 Thread Paolo Patierno
ch I think it's really clear and give us more possibility to evolve this DeletionTarget class if we'll add different ways to specify such target not only offset based. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advis

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-20 Thread Paolo Patierno
Hi all, I have just updated the KIP with your suggestion. I'm going to continue implementation and tests with these changes, waiting for further discussion. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-19 Thread Paolo Patierno
artitions") having a deleteBefore(Long) factory method for a simple creation when you need to delete before the specified offset. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http:/

Re: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-11 Thread Paolo Patierno
Hi all, since I started voting KIP-204 on October 3rd I haven't seen any votes on that. I know you are busy on 1.0.0 release, just as reminder Thanks. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppati

Re: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-10-02 Thread Paolo Patierno
Just as reminder for other committers in order to have other binding votes, for now we have ... binding +1 Ismael Juma +1 Guozhang Wang non binding +1 Mickael Maison +1 Vahid Hashemian Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft A

[VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-10-02 Thread Paolo Patierno
Hi all, I didn't see any further discussion around this KIP, so I'd like to start the vote for it. Just for reference : https://cwiki.apache.org/confluence/display/KAFKA/KIP-204+%3A+adding+records+deletion+operation+to+the+new+Admin+Client+API Thanks, Paolo Patierno Senior Softwar

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-09-28 Thread Paolo Patierno
Hi, maybe we want to start without the delete records policy for now waiting for a real needs. So I'm removing it from the KIP. I hope for more comments on this KIP-204 so that we can start a vote on Monday. Thanks. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft M

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-09-27 Thread Paolo Patierno
Hi, I have just updated the KIP-204 description with the new TopicDeletionPolicy suggested by the KIP-201. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : pao

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2017-09-27 Thread Paolo Patierno
trol could be better (even already happens for topic deletion). 2. I know about the problem of restarting broker due to changes on policies but what do you mean by doing that on the clients ? Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsof

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2017-09-27 Thread Paolo Patierno
Hi Tom, I guess that at "On topic deletion will be applied on topic and message deletion." you meant something like "The TopicDeletionPolicy will be applied on topic and messages deletion". Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on A

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2017-09-27 Thread Paolo Patierno
allowing or not deletion on time base. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno> Blog : De

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-09-26 Thread Paolo Patierno
Hi Tom, as I said in the KIP-201 discussion I'm ok with having a unique DeleteTopicPolicy but then it could be useful having more information then just the topic name; partitions and offset for messages deletion could be useful for a fine grained use cases. Paolo Patierno Senior Sof

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2017-09-26 Thread Paolo Patierno
delete records takes info like partition and specific offset. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolop

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-09-26 Thread Paolo Patierno
them but not the other. In such a policy a user could also check the timestamp related to the offset for allowing or not deletion on time base. Wdyt ? Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno&

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-09-26 Thread Paolo Patierno
Hi Edoardo, I was referring to the KIP-107 where the delete records operation is coming with the authorizer I mentioned. You are referring to KIP-170 ... same digits, inverse order ! Sorry for that ;) Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure &

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-09-26 Thread Paolo Patierno
t suggest a policy as you are saying but using the authorizer mechanism with operation = DELETE and resource = TOPIC. Is my understanding right ? Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twi

Re: [DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-09-26 Thread Paolo Patierno
r a topic name ? Starting from a specific offset ? Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno>

The way for the tools and the Admin Client API usage

2017-09-26 Thread Paolo Patierno
ols in Java using Admin Client API but maybe during this path having a smooth migration just using new Admin Client API in the current Scala tools first (removing Zookeeper calls) could be better. Maybe for committers, in order to review and merge PRs, little ones are better ... What do you thin

Re: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-09-26 Thread Paolo Patierno
ase ? Wdyt ? Thanks. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno> Blog : DevExperience<http:

[jira] [Reopened] (KAFKA-5588) ConsoleConsumer : uselss --new-consumer option

2017-09-26 Thread Paolo Patierno (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5588?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paolo Patierno reopened KAFKA-5588: --- > ConsoleConsumer : uselss --new-consumer opt

[DISCUSS] KIP-204 : adding records deletion operation to the new Admin Client API

2017-09-18 Thread Paolo Patierno
A+adding+records+deletion+operation+to+the+new+Admin+Client+API Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Azure & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/pa

[jira] [Created] (KAFKA-5925) Adding records deletion operation to the new Admin Client API

2017-09-18 Thread Paolo Patierno (JIRA)
Paolo Patierno created KAFKA-5925: - Summary: Adding records deletion operation to the new Admin Client API Key: KAFKA-5925 URL: https://issues.apache.org/jira/browse/KAFKA-5925 Project: Kafka

[jira] [Created] (KAFKA-5919) Delete records command "version" parameter ignored

2017-09-18 Thread Paolo Patierno (JIRA)
Paolo Patierno created KAFKA-5919: - Summary: Delete records command "version" parameter ignored Key: KAFKA-5919 URL: https://issues.apache.org/jira/browse/KAFKA-5919 Project: Kafka

Admin Client : no way to create topic with default partitions and replication factor

2017-09-13 Thread Paolo Patierno
set default values in this case. What do you think about that ? Thanks. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno> Blog : DevExperience<http://paolopatierno.wordpress.com/>

Re: [DISCUSS] KIP-195: AdminClient.increasePartitions

2017-09-08 Thread Paolo Patierno
My 2 cents about naming ... PartitionCount or NumPartitions sound better to me providing an "absolute" value (as today the kafka-topics script work) for an idempotent operation while the NumPartitionsIncrease name sounds to me like the "increment" value. Paolo Pati

Re: KAFKA-5723: Refactor BrokerApiVersionsCommand to use the new AdminClient

2017-09-07 Thread Paolo Patierno
IRAs related to different tools making their PRs Does this plan make sense ? Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.l

Re: KAFKA-5723: Refactor BrokerApiVersionsCommand to use the new AdminClient

2017-09-07 Thread Paolo Patierno
Hi Ismael, first of all thanks for your reply. So as far as I understood having a branch in the Kafka repo could be better for you as committer to validate small PRs from us and not a big one at the end, right ? Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows

Re: KAFKA-5723: Refactor BrokerApiVersionsCommand to use the new AdminClient

2017-09-07 Thread Paolo Patierno
#x27;s not good for us because we can't go forward with development. Let's see what they say, maybe something like that already happened in the past ... Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor T

Re: KAFKA-5723: Refactor BrokerApiVersionsCommand to use the new AdminClient

2017-09-07 Thread Paolo Patierno
Hi committers, as already asked by Viktor on the JIRA yesterday can you give us a feedback/advice on how to move on that ? Thanks ! Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twit

Re: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-09-07 Thread Paolo Patierno
KIP updated to clarify it will be removed in the 2.0.0 version. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.li

Fw: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-09-06 Thread Paolo Patierno
Hi devs, I haven't seen any votes for this since last month. Is there something that should be addressed in the KIP (it didn't have any comments anymore and for this reason I started the vote). Thanks. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP

Re: KAFKA-5723: Refactor BrokerApiVersionsCommand to use the new AdminClient

2017-08-17 Thread Paolo Patierno
. Any feedback is really appreciated :-) Thanks, Paolo Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolo

Re: Command tools : from Scala to Java, from Zookeeper utils to Admin Client API

2017-08-16 Thread Paolo Patierno
. Does the Java re-implementation of the TopicCommand tool need a KIP ? Thanks, Paolo Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopat

[jira] [Created] (KAFKA-5739) Rewrite KStreamPeekTest at processor level avoiding driver usage

2017-08-16 Thread Paolo Patierno (JIRA)
Paolo Patierno created KAFKA-5739: - Summary: Rewrite KStreamPeekTest at processor level avoiding driver usage Key: KAFKA-5739 URL: https://issues.apache.org/jira/browse/KAFKA-5739 Project: Kafka

[jira] [Resolved] (KAFKA-5684) KStreamPrintProcessor as customized KStreamPeekProcessor

2017-08-16 Thread Paolo Patierno (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5684?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Paolo Patierno resolved KAFKA-5684. --- Resolution: Feedback Received > KStreamPrintProcessor as customized KStreamPeekProces

Re: [VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-08-08 Thread Paolo Patierno
Hi Tom, good question. Due to the new version policies, the deprecation will be in the coming 1.0.0 release but then, due to a breaking change on the API, the removal should be on 2.0.0 I guess. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows

[VOTE] KIP-176 : Remove deprecated new-consumer option for tools

2017-08-08 Thread Paolo Patierno
Hi devs, I didn't see any more comments about this KIP. The JIRAs related to the first step (so making --new-consumer as deprecated with warning messages) are merged. I'd like to start a vote for this KIP. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Micros

Jenkins problem with JDK 8 and Scala 2.12 ?

2017-08-08 Thread Paolo Patierno
Failed to fetch from git://github.com/apache/kafka.git I re-tried twice with same results. All is good with JDK 7 and Scala 2.11 Thanks, Paolo Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<h

Re: KStreamPrintTest : no differences in the unit tests

2017-08-03 Thread Paolo Patierno
Hi Damian, I submit the patch as part of this PR https://github.com/apache/kafka/pull/3611 adding a couple of other tests as well. Maybe you can take a look at it :-) Thanks ! Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Micro

KStreamPrintTest : no differences in the unit tests

2017-08-02 Thread Paolo Patierno
the code seems to be the same (some other "final" missing ...). Is there something that my eyes can't see ? Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitt

Re: Command tools : from Scala to Java, from Zookeeper utils to Admin Client API

2017-08-02 Thread Paolo Patierno
... entirely developed in Java. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno>

Re: Command tools : from Scala to Java, from Zookeeper utils to Admin Client API

2017-08-01 Thread Paolo Patierno
have effect on a specific version (tools migration will be done over more releases). Is that right ? Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatiern

Re: Command tools : from Scala to Java, from Zookeeper utils to Admin Client API

2017-08-01 Thread Paolo Patierno
n pushing users to move from old to new tool ... that is good for us. What do you think ? Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Link

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-08-01 Thread Paolo Patierno
o the configs change with the alter topic as well. Last thing on the KIP ... the "timeout" field in the AlterTopicRequest is missing in the table with related description. Thanks Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & Io

Re: Command tools : from Scala to Java, from Zookeeper utils to Admin Client API

2017-08-01 Thread Paolo Patierno
7;s wrong. I'd like to have some input from committers as well to be sure that the way is good about how handling such use cases. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<h

Re: Kafka Streams debugging with "no fluent" API choice

2017-08-01 Thread Paolo Patierno
Hi Damian, changing the print() method for sure needs a KIP but I guess there is some reason we don't know why they decided to not have a fluent API for that. Regarding my JIRA I don't think a KIP is required, it's just internal stuff ... no ? Thanks Paolo Patierno

[jira] [Created] (KAFKA-5684) KStreamPrintProcessor as customized KStreamPeekProcessor

2017-08-01 Thread Paolo Patierno (JIRA)
Paolo Patierno created KAFKA-5684: - Summary: KStreamPrintProcessor as customized KStreamPeekProcessor Key: KAFKA-5684 URL: https://issues.apache.org/jira/browse/KAFKA-5684 Project: Kafka

Re: Kafka Streams debugging with "no fluent" API choice

2017-08-01 Thread Paolo Patierno
with forwardDownStream = false and providing the usage of Serdes. For this I have opened the following JIRA : https://issues.apache.org/jira/browse/KAFKA-5684 What do you think ? Thanks Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azur

Kafka Streams debugging with "no fluent" API choice

2017-08-01 Thread Paolo Patierno
thing I was going to propose before coming across the PR :-)). Why this preference on the first one ? Thanks Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatier

Re: [DISCUSS] KIP-176: Remove deprecated new-consumer option from tools

2017-07-31 Thread Paolo Patierno
https://github.com/apache/kafka/pull/3537 https://github.com/apache/kafka/pull/3555 Thanks Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : pao

Command tools : from Scala to Java, from Zookeeper utils to Admin Client API

2017-07-28 Thread Paolo Patierno
it's something that could be factorized. Thanks, Paolo. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatierno> Linkedin : paolopatierno<http://it.linkedin.com/

Consumer assign vs subscribe influence on __consumer_offset

2017-07-27 Thread Paolo Patierno
use case 2 (in the opposite direction). Conclusion ... it's possible that there is something that I didn't understand about this interaction between consumers in the same group but asking for partitions in two different way (assign vs subscribe). Thanks, Paolo Paolo Patierno Senior S

About "exclude.internal.topics" ... some thoughts

2017-07-27 Thread Paolo Patierno
uot; or something that explains the difference between subscribing using a topic filter (so the property takes an effect) or direct topic name (property does have no effect). What do you think ? Thanks, Paolo. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP o

[jira] [Created] (KAFKA-5643) Using _DUCKTAPE_OPTIONS has no effect on executing tests

2017-07-26 Thread Paolo Patierno (JIRA)
Paolo Patierno created KAFKA-5643: - Summary: Using _DUCKTAPE_OPTIONS has no effect on executing tests Key: KAFKA-5643 URL: https://issues.apache.org/jira/browse/KAFKA-5643 Project: Kafka

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-07-25 Thread Paolo Patierno
that it would be good to consider > if alterTopics would be a sensible way to support all the use cases or if > it's better to have separate APIs. I think it makes sense to have a single > KIP though as they are related and it will be easier to evaluate as a > whole. > >

  1   2   >