Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Apurva Mehta
Yes, you are right. It is worth mentioning since the default value of the retries config is changing. Thanks, Apurva On Thu, Sep 7, 2017 at 10:25 PM, Sumant Tambe wrote: > I added the following: Due to change in the default value of retries from 0 > to MAX_INT and the

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Sumant Tambe
I added the following: Due to change in the default value of retries from 0 to MAX_INT and the existing default value of max.in.flight.request.per.connection==5, reordering becomes a possibility by default. To prevent reordering, set max.in.flight.request.per.connection==1. It does not hurt to

Re: [VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Apurva Mehta
Thanks for the KIP Sumant, +1 from me. That is the most exhaustive 'Rejected Alternatives' section that I have seen :) One minor point: In the compatibility section, your note on 'max.in.flight.requests.per.connection == 5' resulting in out of order delivery is true irrespective of these

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Sumant Tambe
Just did :) On 7 September 2017 at 17:52, Ismael Juma wrote: > Can we please start the vote on this KIP? The KIP must be accepted by next > Wednesday in order to make the cut for 1.0.0. This issue keeps coming up > again and again, and I'd really like to include a fix for

[VOTE] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Sumant Tambe
Hi all, I would like to open the vote for KIP-91: https://cwiki.apache.org/confluence/display/KAFKA/KIP-91+Provide+Intuitive+User+Timeouts+in+The+Producer Thank you all for your input on the kip so far. Regards, Sumant

Build failed in Jenkins: kafka-trunk-jdk8 #1995

2017-09-07 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Include response in request log -- [...truncated 2.51 MB...] org.apache.kafka.streams.integration.GlobalKTableIntegrationTest >

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Apurva Mehta
Thanks for the comments Ismael. I have gone ahead and incorporated all your suggestions in the KIP document. You convinced me on adding max.message.bytes :) Apurva On Thu, Sep 7, 2017 at 6:12 PM, Ismael Juma wrote: > Thanks for the KIP. +1 (binding) from me. A few minor

Re: [VOTE] KIP-131 - Add access to OffsetStorageReader from SourceConnector

2017-09-07 Thread Randall Hauch
The KIP and PR expose the OffsetStorageReader, which is already exposed to the tasks. The OffsetStorageWriter is part of the implementation, but was not and is not exposed thru the API. > On Sep 7, 2017, at 9:04 PM, Gwen Shapira wrote: > > I just re-read the code for the

[GitHub] kafka pull request #3801: MINOR: Include response in request log

2017-09-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3801 ---

[GitHub] kafka pull request #3813: MINOR: Move request/response schemas to the corres...

2017-09-07 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/3813 MINOR: Move request/response schemas to the corresponding object representation This refactor achieves the following: 1. Breaks up the increasingly unmanageable `Protocol` class. 2.

Build failed in Jenkins: kafka-trunk-jdk8 #1994

2017-09-07 Thread Apache Jenkins Server
See Changes: [ismael] MINOR: Always specify the keystore type in system tests -- [...truncated 2.03 MB...] org.apache.kafka.common.security.ssl.SslFactoryTest >

[jira] [Created] (KAFKA-5859) Avoid retaining AbstractRequest in RequestChannel.Request

2017-09-07 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-5859: -- Summary: Avoid retaining AbstractRequest in RequestChannel.Request Key: KAFKA-5859 URL: https://issues.apache.org/jira/browse/KAFKA-5859 Project: Kafka Issue

Re: [DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-07 Thread Gwen Shapira
Thanks for the KIP, Randall. Those are badly needed! Can we have two metrics with record rate per task? One before SMT and one after? We can have cases where we read 5000 rows from JDBC but write 5 to Kafka, or read 5000 records from Kafka and write 5 due to filtering. I think its important to

Re: [VOTE] KIP-131 - Add access to OffsetStorageReader from SourceConnector

2017-09-07 Thread Gwen Shapira
I just re-read the code for the OffsetStorageWriter, and ran into this comment: * Note that this only provides write functionality. This is intentional to ensure stale data is * never read. Offset data should only be read during startup or reconfiguration of a task. By * always serving those

[jira] [Resolved] (KAFKA-5858) consumer.poll() shouldn't throw exception due to deserialization error

2017-09-07 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5858?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5858. Resolution: Duplicate Duplicate of KAFKA-5470, > consumer.poll() shouldn't throw exception due to

[GitHub] kafka pull request #3808: MINOR: Always specify the keystore type in system ...

2017-09-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3808 ---

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Ismael Juma
Thanks for the KIP. +1 (binding) from me. A few minor comments: 1. We should add a note to the backwards compatibility section explaining the impact of throwing DuplicateSequenceException (a new exception) from `send`. As I understand it, it's not an issue, but good to include it in the KIP. 2.

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

2017-09-07 Thread Ismael Juma
Thanks Tom. Thanks for the KIP. A few comments: 1. Does the `PartitionCount` class still make sense given that the method can only increase the number of partitions now? 2,. In `NewTopic`, we have `numPartitions`. Should we keep using that variant and call the method `increaseNumPartitions`? 3.

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Ismael Juma
Can we please start the vote on this KIP? The KIP must be accepted by next Wednesday in order to make the cut for 1.0.0. This issue keeps coming up again and again, and I'd really like to include a fix for 1.0.0. Ismael On Thu, Sep 7, 2017 at 10:01 PM, Apurva Mehta wrote:

[jira] [Created] (KAFKA-5858) consumer.poll() shouldn't throw exception due to deserialization error

2017-09-07 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-5858: -- Summary: consumer.poll() shouldn't throw exception due to deserialization error Key: KAFKA-5858 URL: https://issues.apache.org/jira/browse/KAFKA-5858 Project: Kafka

[jira] [Resolved] (KAFKA-5820) Remove unneeded synchronized keyword in StreamThread

2017-09-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5820?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5820. -- Resolution: Fixed Fix Version/s: 1.0.0 Issue resolved by pull request 3777

[GitHub] kafka pull request #3777: KAFKA-5820 Remove unneeded synchronized keyword in...

2017-09-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3777 ---

[DISCUSS] KIP-196: Add metrics to Kafka Connect framework

2017-09-07 Thread Randall Hauch
Hi everyone. I've created a new KIP to add metrics to the Kafka Connect framework: https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework The KIP approval deadline is looming, so if you're interested in Kafka Connect metrics please review and provide

Jenkins build is back to normal : kafka-trunk-jdk8 #1992

2017-09-07 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #3812: KAFKA-5657 Connect REST API should include the con...

2017-09-07 Thread tedyu
GitHub user tedyu opened a pull request: https://github.com/apache/kafka/pull/3812 KAFKA-5657 Connect REST API should include the connector type when describing a connector Embed the type of connector in ConnectorInfo You can merge this pull request into a Git repository by

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Apurva Mehta
Thanks for the votes! Jason: I updated the KIP so that the messageFormatVersion field is int8. Guozhang: The type of the config field is a purely internal concept. The public API of the KafkaProducer takes a Map type or a Properties type (which is a map of String to String). So

[jira] [Created] (KAFKA-5857) Excessive heap usage on controller node during reassignment

2017-09-07 Thread Raoufeh Hashemian (JIRA)
Raoufeh Hashemian created KAFKA-5857: Summary: Excessive heap usage on controller node during reassignment Key: KAFKA-5857 URL: https://issues.apache.org/jira/browse/KAFKA-5857 Project: Kafka

[GitHub] kafka pull request #3618: KAFKA-5698: Sort processor nodes based on its sub-...

2017-09-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3618 ---

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Guozhang Wang
+1. A quick clarification question regarding the compatibility plan as for "The legacy values for `enable.idempotence` will be interpreted as follows by the new producer: true will mean required, false will mean off." Right now "enable.idempotence" is defined as Type.BOOLEAN while we are likely

Re: [DISCUSS] KIP-170: Enhanced TopicCreatePolicy and introduction of TopicDeletePolicy

2017-09-07 Thread Roger Hoover
Edoardo, thanks for the KIP. I think it's a good idea overall. +1 especially for including Session/Principal in the API. (#2 mentioned by Ismael) Also, the AlterPolicy should get the same info as create + delete (#4). Cheers, Roger On Thu, Sep 7, 2017 at 8:43 AM, Ismael Juma

[jira] [Resolved] (KAFKA-5698) Sort processor node based on name suffix in TopologyDescription.toString()

2017-09-07 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-5698. -- Resolution: Fixed Fix Version/s: 1.0.0 Issue resolved by pull request 3618

Re: [VOTE] KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient

2017-09-07 Thread Guozhang Wang
Hi Tom, The updated part in "AdminClient:electPreferredLeaders()" looks reasonable to me. If there is no objections from the voted committer by end of the day, I think you can mark it as accepted. Guozhang On Wed, Sep 6, 2017 at 7:42 AM, Tom Bentley wrote: >

Re: 1.0.0 KIPs Update

2017-09-07 Thread Guozhang Wang
Actually my bad, there is already a voting thread and you asked people to recast a vote on a small change. On Thu, Sep 7, 2017 at 2:27 PM, Guozhang Wang wrote: > Hi Tom, > > It seems KIP-183 is still in the discussion phase, and voting has not been > started? > > > Guozhang

Re: 1.0.0 KIPs Update

2017-09-07 Thread Guozhang Wang
Hi Tom, It seems KIP-183 is still in the discussion phase, and voting has not been started? Guozhang On Thu, Sep 7, 2017 at 1:13 AM, Tom Bentley wrote: > Would it be possible to add KIP-183 to the list too, please? > > Thanks, > > Tom > > On 6 September 2017 at 22:04,

Re: [VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Jason Gustafson
+1. Thanks for the KIP. One nit: we could use int8 to represent the message format version. That is how it is represented in the messages themselves. -Jason On Thu, Sep 7, 2017 at 1:51 PM, Apurva Mehta wrote: > Hi, > > I'd like to start a vote for KIP-192: > >

Re: [DISCUSS] KIP-183 - Change PreferredReplicaLeaderElectionCommand to use AdminClient

2017-09-07 Thread Jun Rao
Hi, Tom, It seems that it's useful to know whether the leader is balanced for each partition in ElectPreferredLeadersResult, instead of just being attempted? Thanks, Jun On Wed, Sep 6, 2017 at 4:03 PM, Colin McCabe wrote: > On Wed, Sep 6, 2017, at 01:18, Tom Bentley

Re: [VOTE] KIP-131 - Add access to OffsetStorageReader from SourceConnector

2017-09-07 Thread Gwen Shapira
+1 (binding) Looking forward to see how connector implementations use this in practice :) On Thu, Sep 7, 2017 at 3:49 PM Randall Hauch wrote: > I'd like to open the vote for KIP-131: > >

[jira] [Resolved] (KAFKA-3033) Reassigning partition stuck in progress

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3033?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3033. -- Resolution: Duplicate Resolving this as a duplicate of KAFKA-4914. > Reassigning partition stuck in

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Apurva Mehta
I agree with what Ismael said. Having both retries and delivery.timeout.ms is confusing, and thus the goal is to not have a retries option at all once idempotence is fully battle tested and has become the entrenched default. Until that time, it makes sense to expire batch earlier than

[jira] [Resolved] (KAFKA-4138) Producer data write network traffic to kafka brokers is increased for 50%

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4138?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4138. -- Resolution: Cannot Reproduce Closing inactive issue. Similar issues are fixed in newer versions. Pl

[VOTE] KIP-192 : Provide cleaner semantics when idempotence is enabled

2017-09-07 Thread Apurva Mehta
Hi, I'd like to start a vote for KIP-192: https://cwiki.apache.org/confluence/display/KAFKA/KIP-192+%3A+Provide+cleaner+semantics+when+idempotence+is+enabled Thanks, Apurva

[jira] [Resolved] (KAFKA-4190) kafka-reassign-partitions does not report syntax problem in json

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4190. -- Resolution: Duplicate Resolving this duplicated KAFKA-4914. KAFKA-4914 PR is being reviewed now. >

[jira] [Resolved] (KAFKA-4119) Get topic offset with Kafka SSL

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4119?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4119. -- Resolution: Cannot Reproduce Please reopen if the issue still exists. > Get topic offset with Kafka

[jira] [Resolved] (KAFKA-4020) Kafka consumer stop taking messages from kafka server

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4020?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-4020. -- Resolution: Fixed Please reopen if the issue still exists. > Kafka consumer stop taking messages

[jira] [Resolved] (KAFKA-1993) Enable topic deletion as default

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1993?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1993. -- Resolution: Fixed Resolving this as duplicate of KAFKA-5384 > Enable topic deletion as default >

[jira] [Resolved] (KAFKA-3113) Kafka simple consumer inconsistent result

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3113?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-3113. -- Resolution: Cannot Reproduce may be seed broker is different for different topics. simpleconsumer

[jira] [Resolved] (KAFKA-2022) simpleconsumer.fetch(req) throws a java.nio.channels.ClosedChannelException: null exception when the original leader fails instead of being trapped in the fetchResponse

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2022?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2022. -- Resolution: Won't Fix I think we need to catch the exception and retry with a new leader. Pl reopen

Build failed in Jenkins: kafka-trunk-jdk8 #1991

2017-09-07 Thread Apache Jenkins Server
See Changes: [me] KAFKA-4585: Lower the Minimum Required ACL Permission of OffsetFetch -- [...truncated 3.85 MB...] org.apache.kafka.common.config.ConfigDefTest >

[VOTE] KIP-131 - Add access to OffsetStorageReader from SourceConnector

2017-09-07 Thread Randall Hauch
I'd like to open the vote for KIP-131: https://cwiki.apache.org/confluence/display/KAFKA/KIP-131+-+Add+access+to+OffsetStorageReader+from+SourceConnector Thanks to Florian for submitting the KIP and the implementation, and to everyone else that helped review. Best regards, Randall

Jenkins build is back to normal : kafka-trunk-jdk7 #2725

2017-09-07 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-2595) Processor thread dies due to an uncaught NoSuchElementException

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2595?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2595. -- Resolution: Fixed See the discussion in KAFKA-1804 > Processor thread dies due to an uncaught

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

2017-09-07 Thread Viktor Somogyi
Hi Paolo, To me it makes sense, let's do it :). Viktor On Thu, Sep 7, 2017 at 4:37 PM, Paolo Patierno wrote: > So as commented on the KAFKA-5723 JIRA my plan could be : > > > * Using the KAFKA-3268 as umbrella for the others JIRAs related to > tools refactoring > *

[jira] [Resolved] (KAFKA-2623) Kakfa broker not deleting logs after configured retention time properly

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2623?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2623. -- Resolution: Fixed Time-based log retention is enforced in KIP-33. Pl reopen if you think the issue

[GitHub] kafka pull request #3811: KAFKA-5839: Upgrade Guide doc changes for KIP-130

2017-09-07 Thread fhussonnois
GitHub user fhussonnois opened a pull request: https://github.com/apache/kafka/pull/3811 KAFKA-5839: Upgrade Guide doc changes for KIP-130 You can merge this pull request into a Git repository by running: $ git pull https://github.com/fhussonnois/kafka KAFKA-5839

[jira] [Resolved] (KAFKA-2166) Recreation breaks topic-list

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-2166. -- Resolution: Fixed Deletion related issues are fixed in newer versions. Pl reopen if you think the

[GitHub] kafka pull request #3661: KAFKA-4585: Lower the Minimum Required ACL Permiss...

2017-09-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3661 ---

[jira] [Resolved] (KAFKA-1563) High packet rate between brokers in kafka cluster.

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1563?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1563. -- Resolution: Cannot Reproduce Pl reopen if you think the issue still exists > High packet rate

[jira] [Resolved] (KAFKA-1455) Expose ConsumerOffsetChecker as an api instead of being command line only

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1455?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1455. -- Resolution: Fixed This is has been added to Java Admin API. > Expose ConsumerOffsetChecker as an api

[jira] [Resolved] (KAFKA-47) Create topic support and new ZK data structures for intra-cluster replication

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-47?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-47. Resolution: Fixed Closing this umbrella JIRA as all tasks are resolved. > Create topic support and new ZK

[jira] [Resolved] (KAFKA-641) ConsumerOffsetChecker breaks when using dns names x.b.com as opposed to raw public IP for broker.

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-641?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-641. - Resolution: Fixed This is not relevant in newer versions. > ConsumerOffsetChecker breaks when using dns

[jira] [Resolved] (KAFKA-252) Generalize getOffsetsBefore API to a new more general API getLeaderMetadata

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-252?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-252. - Resolution: Fixed These API were handled in newer versions. > Generalize getOffsetsBefore API to a new

[jira] [Resolved] (KAFKA-980) Crash during log recovery can cause full recovery to never run

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-980. - Resolution: Fixed > Crash during log recovery can cause full recovery to never run >

[jira] [Resolved] (KAFKA-986) Topic Consumption Across multiple instances of consumer groups

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-986. - Resolution: Cannot Reproduce > Topic Consumption Across multiple instances of consumer groups >

[jira] [Resolved] (KAFKA-1093) Log.getOffsetsBefore(t, …) does not return the last confirmed offset before t

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1093?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1093. -- Resolution: Invalid relavent part of the code is not available now. So closing this now. >

Kafka Connect corrupts message schema while doing cast transformation

2017-09-07 Thread Plotnikov Artem Igorevich
Hi, there! I have encountered the problem, that Kafka Connect's Cast transformation loses schema information (basically, schema name) while doing type casting. I have reproduced this problem with the following test in org.apache.kafka.connect.transforms.CastTest for current trunk repository

[jira] [Resolved] (KAFKA-1197) Count of bytes or messages of a topic stored in kafka

2017-09-07 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1197?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-1197. -- Resolution: Fixed Size, LogStartOffset, LogEndOffset are exposed as metrics in newer verions. > Count

[GitHub] kafka pull request #3810: KAFKA-5816: [FOLLOW UP] - create ProducedInternal ...

2017-09-07 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3810 KAFKA-5816: [FOLLOW UP] - create ProducedInternal class Create `ProducedInternal` and remove getters from `Produced` You can merge this pull request into a Git repository by running: $ git pull

[jira] [Created] (KAFKA-5856) AdminClient should be able to increase number of partitions

2017-09-07 Thread Tom Bentley (JIRA)
Tom Bentley created KAFKA-5856: -- Summary: AdminClient should be able to increase number of partitions Key: KAFKA-5856 URL: https://issues.apache.org/jira/browse/KAFKA-5856 Project: Kafka Issue

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

2017-09-07 Thread Tom Bentley
Hi Ismael, OK, KIP-195 has been factored out. Regarding the dynamic configs, I personally still think we should have a > specific protocol API for that Can you explain a little more why? With regards to throttling, it would be > worth thinking about a way where the throttling configs can be >

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

2017-09-07 Thread Ted Yu
Tom: Looks good overall. bq. for the topic from the AlterPartitionCountsResult Please align the name of Result with current proposal. Please also fill in JIRA number when you have it. On Thu, Sep 7, 2017 at 9:38 AM, Tom Bentley wrote: > As suggested by Ismael, I've

[DISCUSS] KIP-195: AdminClient.increasePartitions

2017-09-07 Thread Tom Bentley
As suggested by Ismael, I've factored the increasePartitionCounts() API out of KIP-179 out into a separate KIP which hopefully can progress more quickly. https://cwiki.apache.org/confluence/display/KAFKA/KIP-195%3A+AdminClient.increasePartitions If you've looked at KIP-179 in the last few days

[GitHub] kafka pull request #3809: KAFKA-5853: implement WindowedKStream

2017-09-07 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3809 KAFKA-5853: implement WindowedKStream Add the `WindowedKStream` interface and implementation of methods that don't require `Materialized` You can merge this pull request into a Git repository by

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

2017-09-07 Thread Ismael Juma
Hi Tom, It won't be used within Kafka, but it's a public API that can be used by other projects. And the protocol can be used by non-Java clients. So, there is still value in including it. Regarding the dynamic configs, I personally still think we should have a specific protocol API for that.

Re: [VOTE] KIP-189 - Improve principal builder interface and add support for SASL

2017-09-07 Thread Jason Gustafson
I am closing the vote. Here are the totals: Binding: Ismael, Rajini, Jun, (Me) Non-binding: Mayuresh, Manikumar, Mickael Thanks all for the reviews! On Wed, Sep 6, 2017 at 2:22 PM, Jason Gustafson wrote: > Hi All, > > When implementing this, I found that the

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

2017-09-07 Thread Tom Bentley
Hi Ismael, It would be good to get at least some of this into 1.0.0. We could put the increasePartitions() work into another KIP, but it would be an unused AdminClient API in that release. The consumer of this API will be the TopicsCommand when that get refactored to use the AdminClient. That's

Build failed in Jenkins: kafka-trunk-jdk8 #1990

2017-09-07 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-5777; Add ducktape integration for Trogdor -- [...truncated 4.89 MB...]

Re: [DISCUSS] KIP-170: Enhanced TopicCreatePolicy and introduction of TopicDeletePolicy

2017-09-07 Thread Ismael Juma
Hi Tom, Maybe we can discuss that as part of KIP-179. I don't think we should introduce a separate interface for the changes suggested here. Ismael On Tue, Sep 5, 2017 at 7:05 PM, Tom Bentley wrote: > Hi Edoardo, > > KIP-179 will support changing topic configurations

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

2017-09-07 Thread Ismael Juma
Hi Tom, What do you think of moving `increasePartitionsCount` (or `increaseNumPartitions`) to a separate KIP? That is simple enough that we could potentially include it in 1.0.0. I'd be happy to review it. ReassignPartitions is more complex and we can probably aim to include that in the January

[jira] [Created] (KAFKA-5855) records-lag is always zero

2017-09-07 Thread Mohsen Zainalpour (JIRA)
Mohsen Zainalpour created KAFKA-5855: Summary: records-lag is always zero Key: KAFKA-5855 URL: https://issues.apache.org/jira/browse/KAFKA-5855 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-5854) Handle SASL authentication failures as non-retriable exceptions in clients

2017-09-07 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-5854: - Summary: Handle SASL authentication failures as non-retriable exceptions in clients Key: KAFKA-5854 URL: https://issues.apache.org/jira/browse/KAFKA-5854 Project:

[GitHub] kafka pull request #3808: MINOR: Always specify the keystore type in system ...

2017-09-07 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3808 MINOR: Always specify the keystore type in system tests Also throw an exception if a null keystore type is seen in `SecurityStore`. This should never happen. The default keystore type has

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

2017-09-07 Thread Paolo Patierno
So as commented on the KAFKA-5723 JIRA my plan could be : * Using the KAFKA-3268 as umbrella for the others JIRAs related to tools refactoring * Regarding common components needed by developers involved on different tools, I will create a subtask in this JIRA (i.e. related to the new

[jira] [Created] (KAFKA-5853) Add WindowedKStream interface and implemenation

2017-09-07 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5853: - Summary: Add WindowedKStream interface and implemenation Key: KAFKA-5853 URL: https://issues.apache.org/jira/browse/KAFKA-5853 Project: Kafka Issue Type: Sub-task

Jenkins build is back to normal : kafka-trunk-jdk8 #1989

2017-09-07 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-91 Provide Intuitive User Timeouts in The Producer

2017-09-07 Thread Ismael Juma
Good question regarding retries Sumant. A few comments: 1. Defaulting to MAX_INT makes sense in the context of delivery.timeout.ms, but introduces the possibility of reordering with the default max.in.flight of 5. Personally, I think reordering is better than dropping the message altogether (if

Build failed in Jenkins: kafka-trunk-jdk7 #2724

2017-09-07 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-5777; Add ducktape integration for Trogdor -- [...truncated 921.93 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

[GitHub] kafka pull request #3807: KAFKA-5852: Add filter, filterNot, mapValues and M...

2017-09-07 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3807 KAFKA-5852: Add filter, filterNot, mapValues and Materialized to KTable Add overloads of `filter`, `filterNot`, `mapValues` that take `Materialized` as a param to `KTable`. Deprecate overloads using

[jira] [Created] (KAFKA-5852) Add filter, filterNot, mapValues and Materialized to KTable

2017-09-07 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5852: - Summary: Add filter, filterNot, mapValues and Materialized to KTable Key: KAFKA-5852 URL: https://issues.apache.org/jira/browse/KAFKA-5852 Project: Kafka Issue

[GitHub] kafka pull request #3726: KAFKA-5777. Add ducktape integration for Trogdor

2017-09-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3726 ---

[jira] [Resolved] (KAFKA-5777) Add ducktape integration for the Trogdor Fault injection daemon

2017-09-07 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5777?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-5777. --- Resolution: Fixed Fix Version/s: 1.0.0 Issue resolved by pull request 3726

Build failed in Jenkins: kafka-trunk-jdk7 #2723

2017-09-07 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5844; add groupBy(selector, serialized) to Ktable -- [...truncated 921.94 KB...] kafka.server.epoch.LeaderEpochFileCacheTest >

[GitHub] kafka pull request #3802: KAFKA-5844: add groupBy(selector, serialized) to K...

2017-09-07 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3802 ---

[jira] [Resolved] (KAFKA-5844) Add groupBy(KeyValueMapper, Serialized) to KTable

2017-09-07 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5844?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-5844. --- Resolution: Fixed Issue resolved by pull request 3802 [https://github.com/apache/kafka/pull/3802] >

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

2017-09-07 Thread Ismael Juma
No, I'm suggesting that we think how can merge smaller PRs to trunk. Having a separate branch doesn't help as it can diverge from trunk and a committer would be needed to merge rebases, etc. Ismael On Thu, Sep 7, 2017 at 11:25 AM, Paolo Patierno wrote: > Hi Ismael, > >

Build failed in Jenkins: kafka-trunk-jdk8 #1988

2017-09-07 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5650; add StateStoreBuilder interface and implementations -- [...truncated 3.85 MB...] org.apache.kafka.common.security.ssl.SslFactoryTest

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: [VOTE] KIP-188 - Add new metrics to support health checks

2017-09-07 Thread Rajini Sivaram
Hi all, I have added one more metric to KIP-188 to show the current status of broker's ZooKeeper connections. Please let me know if you have any concerns. Hi Jun, I was wondering which is a better group for FetchMessageConversionsPerSec, now that we have MessageConversionsTimeMs at the request

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

2017-09-07 Thread Paolo Patierno
Yes Tom that's true, we could chose one of us as the integration branch owner without involving committers but at same time it could be useful to involve committers for having the PRs evaluated step by step and not only at the end. For sure when committers are really busy for merging it's not

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

2017-09-07 Thread Ismael Juma
I would also add that it would be easier to review if there were smaller PRs than one big PR. So, it may be worth thinking how progress could be made more incrementally. Ismael On Thu, Sep 7, 2017 at 11:17 AM, Tom Bentley wrote: > I can't speak for the committers, but

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

2017-09-07 Thread Tom Bentley
I can't speak for the committers, but there's nothing to stop you submitting PRs against each others branches. It just needs you to agree which of you will host the integration branch. This would be pretty much exactly the same developer experience as of the branch was in the main Kafak repo

  1   2   >