Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-24 Thread Jorge Esteban Quilcate Otoya
Renaming to --by-duration LGTM Not sure about changing it to --shift-by-duration because we could end up with the same redundancy as before with reset: --reset-offsets --reset-to-*. Maybe changing --shift-offset-by to --shift-by 'n' could make it consistent enough? El vie., 24 feb. 2017 a las 6

KIPs to broaden the utilization of Kafka by making it more user friendly

2017-02-24 Thread Werner Daehn
I would have a couple of ideas about Kafka and would love to get your input. If I am totally off, if it is possible today without me noticing or if you can embrace/do disregard the idea - appreciate all your feedback. Statement 1: A typical scenario for Kafka can be coupling on-premise system with

Re: [DISCUSS] KIP-127: Pluggable JAAS LoginModule configuration for SSL

2017-02-24 Thread Christopher Shannon
Does anyone else have any comments or suggestions on this? On Tue, Feb 21, 2017 at 4:05 PM, Christopher Shannon < christopher.l.shan...@gmail.com> wrote: > I should mention another reason I went with adding this enhancement to the > SSL channel instead of SASL_SSL is that as you can see from my s

[jira] [Created] (KAFKA-4799) session timeout during event processing shuts down stream

2017-02-24 Thread Jacob Gur (JIRA)
Jacob Gur created KAFKA-4799: Summary: session timeout during event processing shuts down stream Key: KAFKA-4799 URL: https://issues.apache.org/jira/browse/KAFKA-4799 Project: Kafka Issue Type: B

[GitHub] kafka pull request #2568: Kafka 4198: Fix Race Condition in KafkaServer Shut...

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2568 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4198) Transient test failure: ConsumerBounceTest.testConsumptionWithBrokerFailures

2017-02-24 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4198?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15882534#comment-15882534 ] ASF GitHub Bot commented on KAFKA-4198: --- Github user asfgit closed the pull request

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-24 Thread Rajini Sivaram
I have updated the KIP based on the discussions so far. Regards, Rajini On Thu, Feb 23, 2017 at 11:29 PM, Rajini Sivaram wrote: > Thank you all for the feedback. > > Ismael #1. It makes sense not to throttle inter-broker requests like > LeaderAndIsr etc. The simplest way to ensure that client

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

2017-02-24 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-4198; Fix race condition in KafkaServer.shutdown() -- [...truncated 305.90 KB...] at org.gradle.internal.operations.DefaultBuildOperat

[jira] [Assigned] (KAFKA-4198) Transient test failure: ConsumerBounceTest.testConsumptionWithBrokerFailures

2017-02-24 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4198?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-4198: -- Assignee: Armin Braun > Transient test failure: ConsumerBounceTest.testConsumptionWithBrokerFai

[jira] [Resolved] (KAFKA-4198) Transient test failure: ConsumerBounceTest.testConsumptionWithBrokerFailures

2017-02-24 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4198?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4198. Resolution: Fixed Fix Version/s: 0.10.2.1 0.10.3.0 > Transient test failur

Re: [DISCUSS] KIP-127: Pluggable JAAS LoginModule configuration for SSL

2017-02-24 Thread Rajini Sivaram
Christopher, 1. We currently disable ssl.client.auth for SASL_SSL. This was done at a time when a broker that had two listeners SSL and SASL_SSL using ssl.client.auth=required for SSL and ssl.client.auth=none for SASL_SSL had no way of specifying two different values for the property. With the cha

[GitHub] kafka pull request #2578: MINOR: ConfigKey variable renamed

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2578 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Created] (KAFKA-4800) Streams State transition ASCII diagrams need fixing and polishing

2017-02-24 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-4800: --- Summary: Streams State transition ASCII diagrams need fixing and polishing Key: KAFKA-4800 URL: https://issues.apache.org/jira/browse/KAFKA-4800 Project: Kafka

add contributor

2017-02-24 Thread Eno Thereska
Hello, Sending on behalf on user who wants to contribute. Could we add Clement Valiente to contributor list please? Jira user id is "cvaliente". Thanks Eno

Re: [DISCUSS] KIP-127: Pluggable JAAS LoginModule configuration for SSL

2017-02-24 Thread Ismael Juma
Hi Christopher, Thanks for the KIP. I have two comments: 1. Can you please explain in the KIP (maybe in the Rejected Alternatives section) why a PrincipalBuilder is not good enough for your use case? This is typically what people use to customise authentication for the TLS case. 2. You mention t

[GitHub] kafka pull request #2592: MINOR: fixed javadoc typo in KafkaProducer::partit...

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2592 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Broaden the utilization of Kafka by being more user friendly

2017-02-24 Thread Werner Daehn
I would have a couple of ideas about Kafka and would love to get your input. If I am totally off, if it is possible today without me noticing or if you can embrace/do disregard the idea. Statement 1: A typical scenario for Kafka can be coupling on-premise system with a Kafka based cloud applicatio

[GitHub] kafka pull request #2594: MINOR: add code quality checks to checkstyle.xml. ...

2017-02-24 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/2594 MINOR: add code quality checks to checkstyle.xml. Also add suppressions Add code quality/complexity checks to checkstyle You can merge this pull request into a Git repository by running: $ git pul

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Ismael Juma
Hi Michael, Did you mean that you were happy to compromise to keep it mutable or immutable? You wrote the former, but it sounded from the sentence that it could have been a typo. So, my thoughts on this is that there are a few things to take into account: 1. Semantics 2. Simplicity of use (the co

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-24 Thread Jay Kreps
A couple of quick points: 1. Even though the implementation of this quota is only using io thread time, i think we should call it something like "request-time". This will give us flexibility to improve the implementation to cover network threads in the future and will avoid exposing internal detai

[jira] [Created] (KAFKA-4801) Transient test failure (part 2): ConsumerBounceTest.testConsumptionWithBrokerFailures

2017-02-24 Thread Armin Braun (JIRA)
Armin Braun created KAFKA-4801: -- Summary: Transient test failure (part 2): ConsumerBounceTest.testConsumptionWithBrokerFailures Key: KAFKA-4801 URL: https://issues.apache.org/jira/browse/KAFKA-4801 Proje

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Michael Pearce
Hi On 1, How can you guarantee two separate implemented clients would add the headers in the same order we are not specifying an order at the protocol level (nor should we) with regards to keyA being ordered before keyB? We shouldn’t be expecting keyA to be always set before keyB. On 2, I be

[GitHub] kafka pull request #2595: MINOR: Fix typos in javadoc and code comments

2017-02-24 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/2595 MINOR: Fix typos in javadoc and code comments You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka minor/fix_typos_1702

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-24 Thread Rajini Sivaram
Thanks, Jay. *(1) *The rename from *request.time*.percent to* io.thread*.units for the quota configuration was based on the change from percent to thread-units, since we will need different quota configuration for I/O threads and network threads if we use units. If we agree that *(2)* percent (or

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-24 Thread Matthias J. Sax
I like this! --by-duration and --shift-by -Matthias On 2/24/17 12:57 AM, Jorge Esteban Quilcate Otoya wrote: > Renaming to --by-duration LGTM > > Not sure about changing it to --shift-by-duration because we could end up > with the same redundancy as before with reset: --reset-offsets > --reset

[jira] [Resolved] (KAFKA-4779) Failure in kafka/tests/kafkatest/tests/core/security_rolling_upgrade_test.py

2017-02-24 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4779?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-4779. Resolution: Fixed Fix Version/s: 0.10.2.1 0.10.3.0 Issue resolved by pull

[GitHub] kafka pull request #2589: KAFKA-4779: Fix security upgrade system test to be...

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2589 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4779) Failure in kafka/tests/kafkatest/tests/core/security_rolling_upgrade_test.py

2017-02-24 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15883144#comment-15883144 ] ASF GitHub Bot commented on KAFKA-4779: --- Github user asfgit closed the pull request

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-24 Thread Jorge Esteban Quilcate Otoya
Great! KIP updated. El vie., 24 feb. 2017 a las 18:22, Matthias J. Sax () escribió: > I like this! > > --by-duration and --shift-by > > > -Matthias > > On 2/24/17 12:57 AM, Jorge Esteban Quilcate Otoya wrote: > > Renaming to --by-duration LGTM > > > > Not sure about changing it to --shift-by-du

Re: [VOTE] KIP-122: Add Reset Consumer Group Offsets tooling

2017-02-24 Thread Matthias J. Sax
+1 On 2/23/17 4:46 PM, Jorge Esteban Quilcate Otoya wrote: > Hi All, > > It seems that there is no further concern with the KIP-122. > At this point we would like to start the voting process. > > The KIP can be found here: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-122%3A+Add+Reset+

Re: add contributor

2017-02-24 Thread Jun Rao
Hi, Clement, Thanks for your interest in Kafka. Added you to the contributor list. Jun On Fri, Feb 24, 2017 at 6:10 AM, Eno Thereska wrote: > Hello, > > Sending on behalf on user who wants to contribute. Could we add Clement > Valiente to contributor list please? Jira user id is "cvaliente". >

[jira] [Assigned] (KAFKA-4800) Streams State transition ASCII diagrams need fixing and polishing

2017-02-24 Thread Clemens Valiente (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4800?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Clemens Valiente reassigned KAFKA-4800: --- Assignee: Clemens Valiente > Streams State transition ASCII diagrams need fixing and

Re: [DISCUSS] KIP-127: Pluggable JAAS LoginModule configuration for SSL

2017-02-24 Thread Christopher Shannon
Rajini, If the override can be dropped for SASL_SSL then I have no problem with doing this as SASL_SSL External (basically just TLS authentication). If the configurable callback handlers KIP passes then that would effect this and one of the callback handlers could be an X509 callback handler. Isma

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-02-24 Thread Dong Lin
Hey Jun, I don't think we should allow failed replicas to be re-created on the good disks. Say there are 2 disks and each of them is 51% loaded. If any disk fail, and we allow replicas to be re-created on the other disks, both disks will fail. Alternatively we can disable replica creation if there

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-24 Thread Jun Rao
Hi, Jay, 2. Regarding request.unit vs request.percentage. I started with request.percentage too. The reasoning for request.unit is the following. Suppose that the capacity has been reached on a broker and the admin needs to add a new user. A simple way to increase the capacity is to increase the n

[GitHub] kafka pull request #2560: KAFKA-4494: Reduce startup and rebalance time

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2560 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Resolved] (KAFKA-4494) Significant startup delays in KStreams app

2017-02-24 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4494?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-4494. -- Resolution: Fixed Fix Version/s: 0.10.3.0 Issue resolved by pull request 2560 [https://gi

[jira] [Commented] (KAFKA-4494) Significant startup delays in KStreams app

2017-02-24 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4494?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15883264#comment-15883264 ] ASF GitHub Bot commented on KAFKA-4494: --- Github user asfgit closed the pull request

Re: [DISCUSS] KIP-127: Pluggable JAAS LoginModule configuration for SSL

2017-02-24 Thread Ismael Juma
Hi Christopher, It is possible to retrieve the certificates already. The Principal returned by TransportLayer.peerPrincipal() is a X500Principal if TLS is being used. We could add a method to make things nicer, potentially, but just wanted you to know that it's possible today. I am in favour of k

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-24 Thread Vahid S Hashemian
Hi Jorge, Thanks for the useful KIP. I have a question regarding the proposed "plan" option. The "current offset" and "lag" values of a topic partition are meaningful within a consumer group. In other words, different consumer groups could have different values for these properties of each topi

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Jason Gustafson
The APIs in the current KIP look good to me. Just a couple questions: why does filter not return Headers? Also would it be useful if the key is a regex? On the point of immutability.. One option might be to use a mutable object only when passing the headers through the interceptor chain. I think a

Re: [DISCUSS] KIP-127: Pluggable JAAS LoginModule configuration for SSL

2017-02-24 Thread Christopher Shannon
Sounds good, thanks for the feedback. I will move my KIP to the discarded section for now as the PrincipalBuilder should be sufficient. If for some reason it is not I can revisit this. Chris On Fri, Feb 24, 2017 at 1:37 PM Ismael Juma wrote: > Hi Christopher, > > It is possible to retrieve the

[GitHub] kafka pull request #2584: MINOR: Fix transient failure of testCannotSendToIn...

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2584 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Michael Pearce
Pattern.compile is expensive, and even if cached String.equals is faster than matched. also if we end up with an internal map in future for performance it will be easier to be by key. As all that's needed is to get header by key. With like the other arguements of let's implement simple and then

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

2017-02-24 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-24 Thread Jason Gustafson
+1 from me (duh). Thanks to all the reviewers. The KIP has been much improved because of it! -Jason On Wed, Feb 22, 2017 at 8:48 AM, Ismael Juma wrote: > Great work on the proposal and iterating on it based on community feedback. > As Jun (and others) said, it's likely that minor changes will h

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-24 Thread Sriram Subramanian
+1. Great work in driving this to a consensus. Lots of good constructive conversations. On Fri, Feb 24, 2017 at 11:48 AM, Jason Gustafson wrote: > +1 from me (duh). Thanks to all the reviewers. The KIP has been much > improved because of it! > > -Jason > > On Wed, Feb 22, 2017 at 8:48 AM, Ismael

[jira] [Commented] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15883423#comment-15883423 ] James Cheng commented on KAFKA-4798: Looks like someone fixed it. Closing. > Javadocs

[jira] [Closed] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng closed KAFKA-4798. -- > Javadocs for 0.10.2 are missing from kafka.apache.org > --

[jira] [Updated] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng updated KAFKA-4798: --- Status: Reopened (was: Closed) > Javadocs for 0.10.2 are missing from kafka.apache.org >

[jira] [Commented] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15883427#comment-15883427 ] James Cheng commented on KAFKA-4798: Sorry, reopening. The website title bar for the

[jira] [Updated] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng updated KAFKA-4798: --- Status: Reopened (was: Reopened) > Javadocs for 0.10.2 are missing from kafka.apache.org > --

[jira] [Updated] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng updated KAFKA-4798: --- Attachment: Screen Shot 2017-02-24 at 12.02.29 PM.png Attached screenshot of what the website tabs loo

[jira] [Resolved] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] James Cheng resolved KAFKA-4798. Resolution: Fixed > Javadocs for 0.10.2 are missing from kafka.apache.org >

[jira] [Commented] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15883467#comment-15883467 ] James Cheng commented on KAFKA-4798: Pinging [~ewencp], since you were the release man

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Michael Pearce
I’ve added the methods on the ProducerRecord that will return a new instance of ProducerRecord with modified headers. On 24/02/2017, 19:22, "Michael Pearce" wrote: Pattern.compile is expensive, and even if cached String.equals is faster than matched. also if we end up with an internal map

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-24 Thread Jorge Esteban Quilcate Otoya
Hi Vahid, Thanks for your comments. Check my answers below: El vie., 24 feb. 2017 a las 19:41, Vahid S Hashemian (< vahidhashem...@us.ibm.com>) escribió: > Hi Jorge, > > Thanks for the useful KIP. > > I have a question regarding the proposed "plan" option. > The "current offset" and "lag" values

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-24 Thread Guozhang Wang
Hello folks, Some people have reported that the java doc was missing for this release. We have updated the web site including the java docs for 0102 just now. Thanks, Guozhang On Wed, Feb 22, 2017 at 2:56 PM, James Cheng wrote: > Woohoo! Thanks for running the release, Ewen! > > -James > > >

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-24 Thread Vahid S Hashemian
Thanks Jorge for addressing my question/suggestion. One last thing. I noticed is that in the example you have for the "plan" option ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-122%3A+Add+Reset+Consumer+Group+Offsets+tooling#KIP-122:AddResetConsumerGroupOffsetstooling-ExecutionOptions

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Jason Gustafson
Hey Michael, I didn't actually comment on the new methods for ProducerRecord and ConsumerRecord. If they only save some boilerplate, I'd just as well not have them. Also a couple minor comments: 1. Is the intent of `Headers.filter` to include or exclude the headers matching the key? Can you add

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Michael Pearce
Hi Jason, Sorry I thought this was the agreed compromise to provide an api that avoid boiler plate in return for immutabilty. If not then mutability will be needed as a goal is to have a single clean method call to append/remove a header. Cheers Mike On 24/02/2017, 22:15, "Jason Gustafson" w

[jira] [Updated] (KAFKA-3995) Add a new configuration "enable.compression.ratio.estimation" to the producer config

2017-02-24 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3995?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-3995: Summary: Add a new configuration "enable.compression.ratio.estimation" to the producer config (was:

[jira] [Commented] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15883632#comment-15883632 ] James Cheng commented on KAFKA-4798: And [~guozhang]. See my comment about the SNAPSHO

[GitHub] kafka pull request #2582: MINOR: Fixed Non-Final Close Method + its Duplicat...

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2582 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[GitHub] kafka pull request #2596: MINOR: Ensure consumer calls poll() if requests ar...

2017-02-24 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/2596 MINOR: Ensure consumer calls poll() if requests are outstanding You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka ensure-poll-w

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Jason Gustafson
Hey Michael, I'm not strongly opposed to them; I just don't see a lot of benefit. One thing it would be good to understand is why a consumer interceptor would need to add headers and why a producer interceptor would need to remove them. Maybe we only need the common cases? Thanks, Jason On Fri,

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-02-24 Thread Becket Qin
Hi Jay, Yeah, I got your point. I think there might be a solution which do not require adding a new configuration. We can start from a very conservative compression ratio say 1.0 and lower it very slowly according to the actual compression ratio until we hit a point that we have to split a batch.

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Michael Pearce
We’re trying to make an eco-system for people to be able to use headers, I think we want to ensure some least common features are supported and not limited. Some examples we have already. On consume interceptors a security interceptor may need to take the current header, decrypt the data and

[GitHub] kafka pull request #2579: MINOR: Make it impossible to invoke `Request.body`...

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2579 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-24 Thread Becket Qin
Thanks for the KIP Jorge. I think this is a useful KIP. I haven't read the KIP in detail yet, some comments from a quick review: 1. A glance at it it seems that there is no delete option. At LinkedIn we identified some cases that users want to delete the committed offset of a group. It would be go

[GitHub] kafka pull request #2596: MINOR: Ensure consumer calls poll() if requests ar...

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2596 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-24 Thread Michael Pearce
KIP updated in response to the below comments: > 1. Is the intent of `Headers.filter` to include or exclude the headers > matching the key? Can you add a javadoc to clarify? > 2. The KIP mentions that we will introduce V4 of FetchRequest and V4 of > ProduceRequest.

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-24 Thread Joel Koshy
+1 Apurva just updated the wiki after discussion on the doc. These aren't major so if anyone needs a summary just view the recent edits on the wiki or comments in the google doc (although I think you need to request permissions for that). Thanks, Joel On Fri, Feb 24, 2017 at 11:50 AM, Sriram Su

[GitHub] kafka pull request #2585: MINOR: Fix potential integer overflow and String.f...

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/2585 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Commented] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15883866#comment-15883866 ] Ewen Cheslack-Postava commented on KAFKA-4798: -- [~wushujames] I fixed the ver

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

2017-02-24 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-24 Thread Guozhang Wang
Hello all, Thanks a billion for your votes and comments. We have collected the following votes so far (+1 from myself as well): Binding +1: 9 Gwen Shapira Jay Kreps Becket Qin Jun Rao Ismael Juma Jason Gustafson Sriram Subramanian Joel Koshy Guozhang Wang Binding -1: 0 Non-Bi

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-24 Thread Guozhang Wang
Correction: Non-Binding +0 count is 2. On Fri, Feb 24, 2017 at 7:44 PM, Guozhang Wang wrote: > Hello all, > > Thanks a billion for your votes and comments. We have collected the > following votes so far (+1 from myself as well): > > Binding +1: 9 > > Gwen Shapira > Jay Kreps > Becket Qin >

[jira] [Commented] (KAFKA-4798) Javadocs for 0.10.2 are missing from kafka.apache.org

2017-02-24 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4798?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15884000#comment-15884000 ] Guozhang Wang commented on KAFKA-4798: -- Thanks [~ewencp], and I agree we should have

New subscriber

2017-02-24 Thread Huimin He
My name is Huimin He. I want to subscribe to the mail list. Thanks, Huimin He

Re: Trying to understand design decision about producer ack and min.insync.replicas

2017-02-24 Thread James Cheng
I read the recent Client Survey (https://www.confluent.io/blog/first-annual-state-apache-kafka-client-use-survey/ ). It said that most responders to the survey said that reliability was critical or very important

[jira] [Commented] (KAFKA-1120) Controller could miss a broker state change

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1120?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15884085#comment-15884085 ] James Cheng commented on KAFKA-1120: I'm not sure if this helps, but I figured out how

[jira] [Commented] (KAFKA-3436) Speed up controlled shutdown.

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15884088#comment-15884088 ] James Cheng commented on KAFKA-3436: Is there a chance this can be worked on for 0.10.

[jira] [Commented] (KAFKA-1342) Slow controlled shutdowns can result in stale shutdown requests

2017-02-24 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1342?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15884094#comment-15884094 ] James Cheng commented on KAFKA-1342: [~jjkoshy], [~toddpalino], is it still true that

[jira] [Created] (KAFKA-4802) Support direct ByteBuffer serializers/deserializers in clients

2017-02-24 Thread Matt Sicker (JIRA)
Matt Sicker created KAFKA-4802: -- Summary: Support direct ByteBuffer serializers/deserializers in clients Key: KAFKA-4802 URL: https://issues.apache.org/jira/browse/KAFKA-4802 Project: Kafka Iss