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

2017-07-31 Thread Paolo Patierno
://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 : paolopat

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

2017-07-28 Thread Paolo Patierno
e but 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/in/pa

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

2017-08-02 Thread Paolo Patierno
ly 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> Blog : De

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

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

Re: Clarification on KafkaConsumer manual partition assignment

2017-07-13 Thread Paolo Patierno
Assigning partitions manually has no relation with consumer groups. I mean ... a consumer doesn't need to be part of a consumer group (so specifying group.id) for having a partition assigned manually. From: venkata sastry akella Sent:

New AdmiClient no rack awareness support

2017-07-13 Thread Paolo Patierno
n any case it passes to the AdminUtils only the brokers which satisfy the rack awareness (could be all or part of them). With new AdminClient, the CreateTopicRequest flows to the AdminManager in the broker which seems to use all the available brokers. Is my understanding right ? Thanks, Paolo Patie

[DISCUSS] KIP-176: Remove deprecated new-consumer option in ConsoleConsumer tool

2017-07-17 Thread Paolo Patierno
it as deprecated in the next release (https://issues.apache.org/jira/browse/KAFKA-5599) as suggested by Ismael Juma. The next step should be to vote/accept the KIP and removing the option from the tool in the subsequent release cycle. Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red

Re: From Scala to Java based tools : joptsimple vs argparse4j

2017-07-10 Thread Paolo Patierno
and so on. 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/paolopatierno> Blog : De

Re: From Scala to Java based tools : joptsimple vs argparse4j

2017-07-10 Thread Paolo Patierno
ierno.wordpress.com/> ____ From: Paolo Patierno <ppatie...@live.com> Sent: Monday, July 10, 2017 8:31 AM To: dev@kafka.apache.org Subject: Re: From Scala to Java based tools : joptsimple vs argparse4j Hi Tom, no I have no specific reason but I'd like to know why it was

Re: From Scala to Java based tools : joptsimple vs argparse4j

2017-07-10 Thread Paolo Patierno
evaluating and discussing) around the TopicCommand : https://github.com/apache/kafka/pull/3514 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

Kafka doc : official repo or web site repo ?

2017-07-13 Thread Paolo Patierno
Stream doc I noticed that it seems to be only in the Kafka web site repo and not in the Kafka repo. Can you clarifying me where to submit PRs for doc ? On which side ? Thanks, Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft A

Re: [DISCUSS] KIP-176: Remove deprecated new-consumer option in ConsoleConsumer tool

2017-07-18 Thread Paolo Patierno
are talking about :-) I hope to have a review on that PR soon for the merge ;) 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 : paolopat

Reason why TopicFilter and related classes are deprecated

2017-07-18 Thread Paolo Patierno
? Working on rewriting the TopicCommand tool using the Admin Client API this question came up to my mind. Thanks, Paolo Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/pp

From Scala to Java based tools : joptsimple vs argparse4j

2017-07-10 Thread Paolo Patierno
for the "new" Java based tools. What do you think ? 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<

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

2017-07-20 Thread Paolo Patierno
rmance and ConsumerGroupCommand. I'm also working on a PR for a new-consumer option deprecation as first step for ConsumerPerformance and ConsumerGroupCommand (as already done, pushing a PR<https://github.com/apache/kafka/pull/3537>, for ConsoleConsumer). Thanks, Paolo Paolo Patierno Sen

Re: Reason why TopicFilter and related classes are deprecated

2017-07-20 Thread Paolo Patierno
ic can be a pattern. My understanding is that we can get rid of TopicFilter, Whitelisy and Blacklist even from the TopicCommand. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno<http://twitter.com/ppatie

Re: Kafka doc : official repo or web site repo ?

2017-07-21 Thread Paolo Patierno
. Instead, if I search for the same phrase in the kafka-site repo I can find it in 0110/streams/index.html. So it seems that the doc is in the kafka-site repo but not in the kafka repo ... where am I wrong ? Thanks, Paolo. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP

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

2017-07-25 Thread Paolo Patierno
be better in order to change topic "high level" structure so number of partitions, replication factors and so on. My opinion is that we need separate API because from my point of view they are different settings. Thanks, Paolo. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Mic

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

2017-07-25 Thread Paolo Patierno
JIRA). I also agree 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 > whol

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

2017-07-27 Thread Paolo Patierno
; 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 on Win

Consumer assign vs subscribe influence on __consumer_offset

2017-07-27 Thread Paolo Patierno
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 Software Engineer (IoT) @

Re: 答复: [ANNOUNCE] New Kafka PMC member Ismael Juma

2017-07-06 Thread Paolo Patierno
Congratulations Ismael ... well deserved ! 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/paolop

Re: [ANNOUNCE] Apache Kafka 0.11.0.0 Released

2017-06-29 Thread Paolo Patierno
Thank you all for great contributions and Ismael for driving this release ! 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

Jenkins problem with JDK 8 and Scala 2.12 ?

2017-08-08 Thread Paolo Patierno
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<http://twit

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: 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 Senior Software

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

2017-08-01 Thread Paolo Patierno
rong. 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<http://twit

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

2017-08-01 Thread Paolo Patierno
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/ppatierno> Linkedin

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

2017-08-01 Thread Paolo Patierno
hing 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> Linkedin

KStreamPrintTest : no differences in the unit tests

2017-08-02 Thread Paolo Patierno
n all 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://twitter.com/ppa

Kafka Streams debugging with "no fluent" API choice

2017-08-01 Thread Paolo Patierno
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/ppatierno>

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 Azure Adv

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 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/paolopatie

Re: Test fail reason on doc fix

2017-06-12 Thread Paolo Patierno
Thanks Michal ... I have done in this way but now the test which failed is now passed. The other test which was successful is now aborted (https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5149/). I'll retry with a new "Retest this please" ;) Paolo Patierno Senior Software Eng

Re: Test fail reason on doc fix

2017-06-12 Thread Paolo Patierno
And after two "Retest this please" .. it worked :-) 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

Re: Starting contribution on Apache Kafka project

2017-06-12 Thread Paolo Patierno
Hi, any news on this ? I see a lot of activity for the new release so maybe the previous email was lost. :) Thanks ! Paolo From: Paolo Patierno Sent: Wednesday, 7 June, 11:31 Subject: Starting contribution on Apache Kafka project To: dev@kafka.apache.org Hi Kafka devs, I'd like to start

Re: Open PRs

2017-06-26 Thread Paolo Patierno
Hi Ismael, thanks for replying to Tom's comment because it is the same situation here with some PRs opened even 2 weeks ago. I was pretty sure that the reason was the 0.11.0 release and you have just confirmed it. Looking forward to see the new Kafka version out, pushing more on the next one

Kafka Streams : parallelism related to repartition topic partitions as well

2017-06-20 Thread Paolo Patierno
t 2 (not 4). Do you confirm this ? 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/paolopa

Re: Mirroring documentation improvement

2017-06-23 Thread Paolo Patierno
I'm interested as well ! Thank you ! 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

Re: Mirroring documentation improvement

2017-06-23 Thread Paolo Patierno
Hi Damina, my username is ppatierno ... thank you very much ! 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

Starting contribution on Apache Kafka project

2017-06-07 Thread Paolo Patierno
Hi Kafka devs, I'd like to start contributing on Kafka and as I read here (http://kafka.apache.org/contributing.html) I'm contacting you to be added to the contributor list so that I could pick up easy JIRAs for getting familiar with it. Is that possible ? Thanks ! Paolo. Paolo Patierno

Test fail reason on doc fix

2017-06-12 Thread Paolo Patierno
that. Because it sounds strange to me, because I didn't change any Java/Scala code, what is the way to go to check the failure reason ? Thanks, Paolo. Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft Azure Advisor Twitter : @ppatierno&

Re: Test fail reason on doc fix

2017-06-12 Thread Paolo Patierno
Hi Tom, this is the problem I noticed for this reason asked here. Btw I don't understand the reason for a failure on a simple doc fix (inside the HTML file) :-) Thanks, Paolo Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embedded & IoT Microsoft A

Fix for Kafka Connect documentation web site

2017-06-19 Thread Paolo Patierno
ecause I found the connect documentation page in the docs on the main repo I decided to open the PR there. Am I wrong ? I see a duplication in this case, same page connect.html on both repos. Thanks, Paolo Paolo Patierno Senior Software Engineer (IoT) @ Red Hat Microsoft MVP on Windows Embe

Re: Kafka Streams vs Spark Streaming : reduce by window

2017-06-18 Thread Paolo Patierno
... 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/paolopatierno> Blog : De

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

2017-09-18 Thread Paolo Patierno
ing+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/paolopa

[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 Software Engineer

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

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 : @ppatierno&

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
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: 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
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 Twitter : @ppatierno&

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 on Windows

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

2017-09-07 Thread Paolo Patierno
As 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.linked

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

2017-09-13 Thread Paolo Patierno
lues 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: [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

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

2017-09-26 Thread Paolo Patierno
the 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
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 Software

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
uggest 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://twitter.

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

2017-09-26 Thread Paolo Patierno
? 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> Blog : DevExp

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-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 MVP

The way for the tools and the Admin Client API usage

2017-09-26 Thread Paolo Patierno
a 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 think about this ?

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

2017-09-26 Thread Paolo Patierno
? 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://paolopat

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
for 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-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
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 Microsoft Azure Ad

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

2017-10-23 Thread Paolo Patierno
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 Advisor Twitter : @ppati

Metadata class doesn't "expose" topics with errors

2017-10-23 Thread Paolo Patierno
Is there any specific reason why "topics with errors" 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 Mic

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

2017-10-23 Thread Paolo Patierno
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/paolop

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: [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 Twi

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

2017-10-19 Thread Paolo Patierno
itions") 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://twit

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

2017-10-23 Thread Paolo Patierno
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<http://it

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

2017-10-24 Thread Paolo Patierno
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) @ Red Hat Microsoft MVP

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

2017-10-25 Thread Paolo Patierno
for that. 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-31 Thread Paolo Patierno
:22 PM To: dev@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 <

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

2017-10-31 Thread Paolo Patierno
in On Wed, Oct 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 discuss

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

2017-10-31 Thread Paolo Patierno
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.linkedin.com

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

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: [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

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: [VOTE] KIP-204 : adding records deletion operation to the new Admin Client API

2017-11-10 Thread Paolo Patierno
d then have a field logStartOffset or lowWatermark instead of having to document it via a comment only. Ismael On Tue, Oct 3, 2017 at 6:51 AM, Paolo Patierno <ppatie...@live.com> wrote: > Hi all, > > I didn't see any further discussion around this KIP, so I'd like to start

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

2017-11-10 Thread Paolo Patierno
s 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 : @ppatierno<h

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-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 : @ppatierno&

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

2017-11-13 Thread Paolo Patierno
Monday, November 13, 2017 9:21:07 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 cas

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 Engineer (IoT) @ Red

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 : De

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:

[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

[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

  1   2   >