[GitHub] kafka pull request #3406: KAFKA-5490: Retain empty batch for last sequence o...

2017-06-21 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/3406 KAFKA-5490: Retain empty batch for last sequence of each producer You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-5490

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-21 Thread Guozhang Wang
Thanks for the updated KIP, some more comments: 1.The config name is "default.deserialization.exception.handler" while the interface class name is "RecordExceptionHandler", which is more general than the intended purpose. Could we rename the class name accordingly? 2. Could you describe the full

Re: [DISCUSS] KIP-167: Add a restoreAll method to StateRestoreCallback

2017-06-21 Thread Guozhang Wang
More specifically, if we can replace the first parameter from the String store name to the store instance itself, would that be sufficient to cover ` StateRestoreNotification`? On Wed, Jun 21, 2017 at 7:13 PM, Guozhang Wang wrote: > Bill, > > I'm wondering why we need the

Re: [DISCUSS] KIP-167: Add a restoreAll method to StateRestoreCallback

2017-06-21 Thread Guozhang Wang
Bill, I'm wondering why we need the `StateRestoreNotification` while still having `StateRestoreListener`, could the above setup achievable just with `StateRestoreListener.onRestoreStart / onRestoreEnd`? I.e. it seems the later can subsume any use cases intended for the former API. Guozhang On

Re: Handling 2 to 3 Million Events before Kafka

2017-06-21 Thread Garrett Barton
Getting good concurrency in a webapp is more than doable. Check out these benchmarks: https://www.techempower.com/benchmarks/#section=data-r14=ph=db I linked to the single query one because thats closest to a single operation like you will be doing. I'd also note if the data delivery does not

[GitHub] kafka pull request #3405: KAFKA-5495: Update docs to use `kafka-consumer-gro...

2017-06-21 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/3405 KAFKA-5495: Update docs to use `kafka-consumer-groups.sh` for checking consumer offsets And remove the deprecated `ConsumerOffsetChecker` example. You can merge this pull request into a Git

[GitHub] kafka pull request #3387: MINOR: Update documentation to use `kafka-consumer...

2017-06-21 Thread vahidhashemian
Github user vahidhashemian closed the pull request at: https://github.com/apache/kafka/pull/3387 --- 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

[GitHub] kafka pull request #3375: KAFKA-5474: Streams StandbyTask should no checkpoi...

2017-06-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3375 --- 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

[jira] [Created] (KAFKA-5495) Replace the deprecated 'ConsumerOffsetChecker' in documentation

2017-06-21 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-5495: -- Summary: Replace the deprecated 'ConsumerOffsetChecker' in documentation Key: KAFKA-5495 URL: https://issues.apache.org/jira/browse/KAFKA-5495 Project: Kafka

[GitHub] kafka pull request #3404: KAFKA-5476: Implement a system test that creates n...

2017-06-21 Thread cmccabe
GitHub user cmccabe opened a pull request: https://github.com/apache/kafka/pull/3404 KAFKA-5476: Implement a system test that creates network partitions KAFKA-5476: Implement a system test that creates network partitions You can merge this pull request into a Git repository by

Re: [DISCUSS] KIP-168: Add TotalTopicCount metric per cluster

2017-06-21 Thread Abhishek Mendhekar
Hi Dong, Thanks for the suggestion! I think TopicCount sounds reasonable to me and it definitely seems consistent with the other metric names. I will update the proposal to reflect this change. Thanks, Abhishek On Wed, Jun 21, 2017 at 2:17 PM, Dong Lin wrote: > Hey

[jira] [Created] (KAFKA-5494) Idempotent producer should not require max.in.flight.requests.per.connection=1 and acks=all

2017-06-21 Thread Apurva Mehta (JIRA)
Apurva Mehta created KAFKA-5494: --- Summary: Idempotent producer should not require max.in.flight.requests.per.connection=1 and acks=all Key: KAFKA-5494 URL: https://issues.apache.org/jira/browse/KAFKA-5494

[GitHub] kafka pull request #3398: allow transactions in producer perf script

2017-06-21 Thread tcrayford
Github user tcrayford closed the pull request at: https://github.com/apache/kafka/pull/3398 --- 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

[jira] [Created] (KAFKA-5493) Optimize calls to flush for tasks and standby tasks

2017-06-21 Thread Bill Bejeck (JIRA)
Bill Bejeck created KAFKA-5493: -- Summary: Optimize calls to flush for tasks and standby tasks Key: KAFKA-5493 URL: https://issues.apache.org/jira/browse/KAFKA-5493 Project: Kafka Issue Type:

[GitHub] kafka pull request #3400: KAFKA-5491: Enable transactions in ProducerPerform...

2017-06-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3400 --- 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

[jira] [Resolved] (KAFKA-5491) The ProducerPerformance tool should support transactions

2017-06-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5491?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-5491. Resolution: Fixed Fix Version/s: 0.11.0.0 Issue resolved by pull request 3400

[GitHub] kafka pull request #3403: MINOR: Turn off caching in demos for more understa...

2017-06-21 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/3403 MINOR: Turn off caching in demos for more understandable outputs You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka

[GitHub] kafka pull request #3399: KAFKA-5475: Connector config validation should inc...

2017-06-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3399 --- 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

Re: [DISCUSS] KIP-168: Add TotalTopicCount metric per cluster

2017-06-21 Thread Dong Lin
Hey Abhishek, I think the metric is useful. Sorry for being late on this. I am wondering if TopicCount is a better name than TotalTopicCount, given that we currently have metric with names OfflinePartitionsCount, LeaderCount, PartitionCount etc. Thanks, Dong On Fri, Jun 16, 2017 at 9:09 AM,

[GitHub] kafka pull request #3373: MINOR: Detail message/batch size implications for ...

2017-06-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3373 --- 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

[jira] [Created] (KAFKA-5492) LogRecoveryTest.testHWCheckpointWithFailuresSingleLogSegment transient failure

2017-06-21 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5492: -- Summary: LogRecoveryTest.testHWCheckpointWithFailuresSingleLogSegment transient failure Key: KAFKA-5492 URL: https://issues.apache.org/jira/browse/KAFKA-5492

[GitHub] kafka pull request #3402: KAFKA-5486: org.apache.kafka logging should go to ...

2017-06-21 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/3402 KAFKA-5486: org.apache.kafka logging should go to server.log You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka

[VOTE] KIP-168: Add TotalTopicCount metric per cluster

2017-06-21 Thread Abhishek Mendhekar
Hi Kafka Dev, I did like to start the voting on - https://cwiki.apache.org/confluence/display/KAFKA/KIP- 168%3A+Add+TotalTopicCount+metric+per+cluster Discussions will continue on -

Re: [DISCUSS] KIP-163: Lower the Minimum Required ACL Permission of OffsetFetch

2017-06-21 Thread Vahid S Hashemian
I appreciate everyone's feedback so far on this KIP. Before starting a vote, I'd like to also ask for feedback on the "Additional Food for Thought" section in the KIP:

[GitHub] kafka pull request #3083: KAFKA-1955: [WIP] Disk based buffer in Producer

2017-06-21 Thread blbradley
Github user blbradley closed the pull request at: https://github.com/apache/kafka/pull/3083 --- 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

[jira] [Created] (KAFKA-5491) The ProducerPerformance tool should support transactions

2017-06-21 Thread Apurva Mehta (JIRA)
Apurva Mehta created KAFKA-5491: --- Summary: The ProducerPerformance tool should support transactions Key: KAFKA-5491 URL: https://issues.apache.org/jira/browse/KAFKA-5491 Project: Kafka Issue

Re: [VOTE] 0.11.0.0 RC1

2017-06-21 Thread Tom Crayford
That looks better than mine, nice! I think the tooling matters a lot to the usability of the product we're shipping, being able to test out Kafka's features on your own hardware/setup is very important to knowing if it can work. On Wed, Jun 21, 2017 at 8:01 PM, Apurva Mehta

Re: [VOTE] 0.11.0.0 RC1

2017-06-21 Thread Apurva Mehta
Hi Tom, I actually made modifications to the produce performance tool to do real transactions earlier this week as part of our benchmarking (results published here: bit.ly/kafka-eos-perf). I just submitted that patch here: https://github.com/apache/kafka/pull/3400/files I think my version is

[jira] [Created] (KAFKA-5490) Deletion of tombstones during cleaning should consider idempotent message retention

2017-06-21 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5490: -- Summary: Deletion of tombstones during cleaning should consider idempotent message retention Key: KAFKA-5490 URL: https://issues.apache.org/jira/browse/KAFKA-5490

[GitHub] kafka pull request #3401: MINOR: explain producer naming within Streams

2017-06-21 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/3401 MINOR: explain producer naming within Streams You can merge this pull request into a Git repository by running: $ git pull https://github.com/mjsax/kafka minor-producer-naming-011

[GitHub] kafka pull request #3400: Enable transactions in ProducerPerformance Tool

2017-06-21 Thread apurvam
GitHub user apurvam opened a pull request: https://github.com/apache/kafka/pull/3400 Enable transactions in ProducerPerformance Tool With this patch, the `ProducePerfomance` tool can create transactions of differing durations. This patch was used to to collect the initial

[jira] [Created] (KAFKA-5489) Failing test: InternalTopicIntegrationTest.shouldCompactTopicsForStateChangelogs

2017-06-21 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5489: -- Summary: Failing test: InternalTopicIntegrationTest.shouldCompactTopicsForStateChangelogs Key: KAFKA-5489 URL: https://issues.apache.org/jira/browse/KAFKA-5489

Re: confluence permission request

2017-06-21 Thread Jeremy Hanna
May I also get permission to modify the Kafka Confluence space? My user is jeromatron and I’d like to help with documentation. > On Jun 21, 2017, at 11:22 AM, Damian Guy wrote: > > Hi, > > That should be done. > > Thanks, > Damian > > On Wed, 21 Jun 2017 at 05:42

RE: Handling 2 to 3 Million Events before Kafka

2017-06-21 Thread Tauzell, Dave
I’m not really familiar with Netty so I won’t be of much help. Maybe try posting on a Netty forum to see what they think? -Dave From: SenthilKumar K [mailto:senthilec...@gmail.com] Sent: Wednesday, June 21, 2017 10:28 AM To: Tauzell, Dave Cc: us...@kafka.apache.org; senthilec...@apache.org;

[GitHub] kafka pull request #3394: KAFKA-5475: Connector config validation needs to i...

2017-06-21 Thread kkonstantine
Github user kkonstantine closed the pull request at: https://github.com/apache/kafka/pull/3394 --- 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

Re: [VOTE] 0.11.0.0 RC1

2017-06-21 Thread Tom Crayford
Hi there, I'm -1 (non-binding) on shipping this RC. Heroku has carried on performance testing with 0.11 RC1. We have updated our test setup to use 0.11.0.0 RC1 client libraries. Without any of the transactional features enabled, we get slightly better performance than 0.10.2.1 with 10.2.1 client

[GitHub] kafka pull request #3399: KAFKA-5475: Connector config validation should inc...

2017-06-21 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/3399 KAFKA-5475: Connector config validation should include fields for defined transformation aliases You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka pull request #3377: KAFKA-5477: Lower retryBackoff for AddPartitionsRe...

2017-06-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3377 --- 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

[GitHub] kafka pull request #3398: allow transactions in producer perf script

2017-06-21 Thread tcrayford
GitHub user tcrayford opened a pull request: https://github.com/apache/kafka/pull/3398 allow transactions in producer perf script allow the transactional producer to be enabled in `producer-perf.sh`, with a new flag `--use-transactions` You can merge this pull request into a Git

[GitHub] kafka pull request #3397: KAFKA-5413: Port fix to 0.10.2 branch

2017-06-21 Thread kelvinrutt
GitHub user kelvinrutt opened a pull request: https://github.com/apache/kafka/pull/3397 KAFKA-5413: Port fix to 0.10.2 branch Port KAFKA-5413 to the 0.10.2 branch You can merge this pull request into a Git repository by running: $ git pull https://github.com/kelvinrutt/kafka

Re: Contributor

2017-06-21 Thread Maisnam Ns
Can you please add me also (username: niranjanmaisnam) to the contributors list for this project at issues.apache.org? (kafka) Thank you, niranjan On Wed, Jun 21, 2017 at 10:08 PM, Tom Bentley wrote: > Thanks! > > On 21 Jun 2017 4:20 pm, "Damian Guy"

Re: Contributor

2017-06-21 Thread Tom Bentley
Thanks! On 21 Jun 2017 4:20 pm, "Damian Guy" wrote: > Done - thanks > > On Wed, 21 Jun 2017 at 12:19 Tom Bentley wrote: > > > Please can I also be added? My username is tombentley. > > > > Thanks > > > > Tom > > > > On 21 June 2017 at 12:03, Damian

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Eno Thereska
To make it clear, it’s outlined by Damian, I just copy pasted what he told me in person :) Eno > On Jun 21, 2017, at 4:40 PM, Bill Bejeck wrote: > > +1 for the approach outlined above by Eno. > > On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy wrote: >

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Bill Bejeck
+1 for the approach outlined above by Eno. On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy wrote: > Thanks Eno. > > Yes i agree. We could apply this same approach to most of the operations > where we have multiple overloads, i.e., we have a single method for each > operation

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Damian Guy
Thanks Eno. Yes i agree. We could apply this same approach to most of the operations where we have multiple overloads, i.e., we have a single method for each operation that takes the required parameters and everything else is specified as you have done above. On Wed, 21 Jun 2017 at 16:24 Eno

Re: Handling 2 to 3 Million Events before Kafka

2017-06-21 Thread SenthilKumar K
So netty would work for this case ? I do have netty server and seems to be i'm not getting the expected results .. here is the git https://github.com/senthilec566/netty4-server , is this right implementation ? Cheers, Senthil On Wed, Jun 21, 2017 at 7:45 PM, Tauzell, Dave

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Eno Thereska
(cc’ing user-list too) Given that we already have StateStoreSuppliers that are configurable using the fluent-like API, probably it’s worth discussing the other examples with joins and serdes first since those have many overloads and are in need of some TLC. So following your example, I guess

Re: confluence permission request

2017-06-21 Thread Damian Guy
Hi, That should be done. Thanks, Damian On Wed, 21 Jun 2017 at 05:42 Kenji Hayashida wrote: > To Kafka Dev Team, > > Sorry, forgot sending my ID. > My ID is kenjih. > > Thanks. > > - Kenji Hayashida > > 2017-06-21 13:29 GMT+09:00 Kenji Hayashida : >

Re: Contributor

2017-06-21 Thread Damian Guy
Done - thanks On Wed, 21 Jun 2017 at 12:19 Tom Bentley wrote: > Please can I also be added? My username is tombentley. > > Thanks > > Tom > > On 21 June 2017 at 12:03, Damian Guy wrote: > > > Hi Andras, > > > > You should have access now. > > > >

[DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Damian Guy
Hi, I'd like to get a discussion going around some of the API choices we've made in the DLS. In particular those that relate to stateful operations (though this could expand). As it stands we lean heavily on overloaded methods in the API, i.e, there are 9 overloads for KGroupedStream.count(..)!

Consumer Lag Metric in Kafka

2017-06-21 Thread Madhav Ancha (BLOOMBERG/ 919 3RD A)
Hi, When producerOffset is used in calculating ConsumerLag/MaxLag metric for a consumer, is the producerOffset a) the partition offset that is visible to the clients at the leader b) or the partition offset that is waiting to be replicated at the leader please. Thanks Madhav.

RE: Handling 2 to 3 Million Events before Kafka

2017-06-21 Thread Tauzell, Dave
I see. 1. You don’t want the 100k machines sending directly to kafka. 2. You can only have a small number of web servers People certainly have web-servers handling over 100k concurrent connections. See this for some examples: https://github.com/smallnest/C1000K-Servers . It

Re: Handling 2 to 3 Million Events before Kafka

2017-06-21 Thread SenthilKumar K
Thanks Jeyhun. Yes http server would be problematic here w.r.t network , memory .. Hi Dave , The problem is not with Kafka , it's all about how do you handle huge data before kafka. I did a simple test with 5 node Kafka Cluster which gives good result ( ~950 MB/s ) ..So Kafka side i dont see a

[GitHub] kafka pull request #3396: KAFKA-4931: stop script fails due 4096 ps output l...

2017-06-21 Thread tombentley
GitHub user tombentley opened a pull request: https://github.com/apache/kafka/pull/3396 KAFKA-4931: stop script fails due 4096 ps output limit This also fixes KAFKA-4389 and KAFKA-4297, which were exactly the same issue but for kafka-server-stop.sh. You can merge this pull

[jira] [Created] (KAFKA-5488) KStream.branch should not return a Array of streams we have to access by known index

2017-06-21 Thread Marcel Silberhorn (JIRA)
Marcel Silberhorn created KAFKA-5488: Summary: KStream.branch should not return a Array of streams we have to access by known index Key: KAFKA-5488 URL: https://issues.apache.org/jira/browse/KAFKA-5488

RE: Handling 2 to 3 Million Events before Kafka

2017-06-21 Thread Tauzell, Dave
What are your configurations? - production - brokers - consumers Is the problem that web servers cannot send to Kafka fast enough or your consumers cannot process messages off of kafka fast enough? What is the average size of these messages? -Dave -Original Message- From: SenthilKumar

Re: Handling 2 to 3 Million Events before Kafka

2017-06-21 Thread Jeyhun Karimov
Hi, With kafka you can increase overall throughput by increasing the number of nodes in a cluster. I had a similar issue, where we needed to ingest vast amounts of data to streaming system. In our case, kafka was a bottleneck, because of disk I/O. To solve it, we implemented (simple) distributed

Handling 2 to 3 Million Events before Kafka

2017-06-21 Thread SenthilKumar K
Hi Team , Sorry if this question is irrelevant to Kafka Group ... I have been trying to solve problem of handling 5 GB/sec ingestion. Kafka is really good candidate for us to handle this ingestion rate .. 100K machines > { Http Server (Jetty/Netty) } --> Kafka Cluster.. I see the problem

[jira] [Created] (KAFKA-5487) Rolling upgrade test for streams

2017-06-21 Thread Eno Thereska (JIRA)
Eno Thereska created KAFKA-5487: --- Summary: Rolling upgrade test for streams Key: KAFKA-5487 URL: https://issues.apache.org/jira/browse/KAFKA-5487 Project: Kafka Issue Type: Bug

Re: Contributor

2017-06-21 Thread Tom Bentley
Please can I also be added? My username is tombentley. Thanks Tom On 21 June 2017 at 12:03, Damian Guy wrote: > Hi Andras, > > You should have access now. > > Thanks, > Damian > > On Wed, 21 Jun 2017 at 10:45 Andras Beni wrote: > > > Hi All, > >

[GitHub] kafka pull request #3395: KAFKA-3575: Use console consumer access topic that...

2017-06-21 Thread tombentley
GitHub user tombentley opened a pull request: https://github.com/apache/kafka/pull/3395 KAFKA-3575: Use console consumer access topic that does not exist, ca… …n not use "Control + C" to exit process A finally block is not guaranteed to execute in the event of Ctrl+C

Re: Contributor

2017-06-21 Thread Damian Guy
Hi Andras, You should have access now. Thanks, Damian On Wed, 21 Jun 2017 at 10:45 Andras Beni wrote: > Hi All, > > I'd like to contribute to Apache Kafka. > Can you please add me (username: andrasbeni) to the contributors list for > this project at issues.apache.org?

[GitHub] kafka pull request #3289: MINOR: add Yahoo benchmark to nightly runs

2017-06-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3289 --- 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

[GitHub] kafka pull request #3394: KAFKA-5475: Connector config validation needs to i...

2017-06-21 Thread kkonstantine
GitHub user kkonstantine opened a pull request: https://github.com/apache/kafka/pull/3394 KAFKA-5475: Connector config validation needs to include tranformation types You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka pull request #3393: KAFKA-5319: Add a tool to balance replicas and lea...

2017-06-21 Thread MarkTcMA
GitHub user MarkTcMA opened a pull request: https://github.com/apache/kafka/pull/3393 KAFKA-5319: Add a tool to balance replicas and leaders of cluster You can merge this pull request into a Git repository by running: $ git pull https://github.com/MarkTcMA/kafka KAFKA-5319

[GitHub] kafka pull request #3392: KAFKA-5319:Add a tool to balance the cluster

2017-06-21 Thread MarkTcMA
Github user MarkTcMA closed the pull request at: https://github.com/apache/kafka/pull/3392 --- 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

[GitHub] kafka pull request #3392: KAFKA-5319:Add a tool to balance the cluster

2017-06-21 Thread MarkTcMA
GitHub user MarkTcMA opened a pull request: https://github.com/apache/kafka/pull/3392 KAFKA-5319:Add a tool to balance the cluster It is the code about

[jira] [Created] (KAFKA-5486) org.apache.kafka logging should go to server.log

2017-06-21 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-5486: -- Summary: org.apache.kafka logging should go to server.log Key: KAFKA-5486 URL: https://issues.apache.org/jira/browse/KAFKA-5486 Project: Kafka Issue Type: Bug

Contributor

2017-06-21 Thread Andras Beni
Hi All, I'd like to contribute to Apache Kafka. Can you please add me (username: andrasbeni) to the contributors list for this project at issues.apache.org? Thank you, Andras

Re: [DISCUSS]: KIP-161: streams record processing exception handlers

2017-06-21 Thread Eno Thereska
Thanks Guozhang, I’ve updated the KIP and hopefully addressed all the comments so far. In the process also changed the name of the KIP to reflect its scope better: https://cwiki.apache.org/confluence/display/KAFKA/KIP-161%3A+streams+deserialization+exception+handlers

[GitHub] kafka pull request #3378: MINOR: explain producer naming within Streams

2017-06-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3378 --- 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

Jenkins build is back to normal : kafka-0.10.2-jdk7 #175

2017-06-21 Thread Apache Jenkins Server
See

[GitHub] kafka pull request #3136: KAFKA-5319 Add a tool to make cluster replica and ...

2017-06-21 Thread MarkTcMA
Github user MarkTcMA closed the pull request at: https://github.com/apache/kafka/pull/3136 --- 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