Re: [DISCUSS] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Ismael Juma
Thanks for the KIP Mickael. I suggest starting a vote.

Ismael

On Mon, Aug 21, 2017 at 2:51 PM, Mickael Maison 
wrote:

> Hi all,
>
> I have created a KIP to cleanup the way client-ids are handled by
> brokers and clients.
>
> Currently the Java clients have some restrictions on the client-ids
> that are not enforced by the brokers. Using 3rd party clients,
> client-ids containing any characters can be used causing some strange
> behaviours in the way brokers handle metrics and quotas.
>
> Feedback is appreciated.
>
> Thanks
>


[VOTE] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Mickael Maison
Hi,

I'd like to start the vote on KIP-190:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-190%3A+Handle+client-ids+consistently+between+clients+and+brokers

Thanks


Re: [VOTE] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Ismael Juma
Thanks for the KIP. +1 (binding) from me.

Because this is a bug fix, the change is simple and the impact can be bad,
I suggest considering this for the 1.0.0 release. Guozhang's call though.

Ismael

On Thu, Sep 14, 2017 at 9:52 AM, Mickael Maison 
wrote:

> Hi,
>
> I'd like to start the vote on KIP-190:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 190%3A+Handle+client-ids+consistently+between+clients+and+brokers
>
> Thanks
>


Re: [DISCUSS] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Tom Bentley
Hi Mickael,

I was just wondering why the restriction was imposed for Java clients the
first place, do you know?

Cheers,

Tom

On 14 September 2017 at 09:16, Ismael Juma  wrote:

> Thanks for the KIP Mickael. I suggest starting a vote.
>
> Ismael
>
> On Mon, Aug 21, 2017 at 2:51 PM, Mickael Maison 
> wrote:
>
> > Hi all,
> >
> > I have created a KIP to cleanup the way client-ids are handled by
> > brokers and clients.
> >
> > Currently the Java clients have some restrictions on the client-ids
> > that are not enforced by the brokers. Using 3rd party clients,
> > client-ids containing any characters can be used causing some strange
> > behaviours in the way brokers handle metrics and quotas.
> >
> > Feedback is appreciated.
> >
> > Thanks
> >
>


[GitHub] kafka pull request #3795: KAFKA-5783: Add KafkaPrincipalBuilder with support...

2017-09-14 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3795


---


Re: [VOTE] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Edoardo Comar
+1 (non-binding)
--

Edoardo Comar

IBM Message Hub

IBM UK Ltd, Hursley Park, SO21 2JN



From:   Ismael Juma 
To: dev@kafka.apache.org
Date:   14/09/2017 09:56
Subject:Re: [VOTE] KIP-190: Handle client-ids consistently between 
clients and brokers
Sent by:isma...@gmail.com



Thanks for the KIP. +1 (binding) from me.

Because this is a bug fix, the change is simple and the impact can be bad,
I suggest considering this for the 1.0.0 release. Guozhang's call though.

Ismael

On Thu, Sep 14, 2017 at 9:52 AM, Mickael Maison 
wrote:

> Hi,
>
> I'd like to start the vote on KIP-190:
> 
https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_confluence_display_KAFKA_KIP-2D&d=DwIBaQ&c=jf_iaSHvJObTbx-siA1ZOg&r=EzRhmSah4IHsUZVekRUIINhltZK7U0OaeRo7hgW4_tQ&m=6-i-zyk332E_rCIPJN_-IvBSuVdiMjzXzl0BUQFQmWU&s=WUK0rNOyKKGFJFCQ5DF0VnfaceASTLdRZSwqDXK3x3Q&e=
 

> 190%3A+Handle+client-ids+consistently+between+clients+and+brokers
>
> Thanks
>



Unless stated otherwise above:
IBM United Kingdom Limited - Registered in England and Wales with number 
741598. 
Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU


[GitHub] kafka pull request #2546: KAFKA-4764: Improve diagnostics for SASL auth fail...

2017-09-14 Thread rajinisivaram
Github user rajinisivaram closed the pull request at:

https://github.com/apache/kafka/pull/2546


---


Re: [VOTE] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Rajini Sivaram
+1 (binding)

Thank you for the KIP, Mickael.

On Thu, Sep 14, 2017 at 10:24 AM, Edoardo Comar  wrote:

> +1 (non-binding)
> --
>
> Edoardo Comar
>
> IBM Message Hub
>
> IBM UK Ltd, Hursley Park, SO21 2JN
>
>
>
> From:   Ismael Juma 
> To: dev@kafka.apache.org
> Date:   14/09/2017 09:56
> Subject:Re: [VOTE] KIP-190: Handle client-ids consistently between
> clients and brokers
> Sent by:isma...@gmail.com
>
>
>
> Thanks for the KIP. +1 (binding) from me.
>
> Because this is a bug fix, the change is simple and the impact can be bad,
> I suggest considering this for the 1.0.0 release. Guozhang's call though.
>
> Ismael
>
> On Thu, Sep 14, 2017 at 9:52 AM, Mickael Maison 
> wrote:
>
> > Hi,
> >
> > I'd like to start the vote on KIP-190:
> >
> https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.
> apache.org_confluence_display_KAFKA_KIP-2D&d=DwIBaQ&c=jf_
> iaSHvJObTbx-siA1ZOg&r=EzRhmSah4IHsUZVekRUIINhltZK7U0
> OaeRo7hgW4_tQ&m=6-i-zyk332E_rCIPJN_-IvBSuVdiMjzXzl0BUQFQmWU&s=
> WUK0rNOyKKGFJFCQ5DF0VnfaceASTLdRZSwqDXK3x3Q&e=
>
> > 190%3A+Handle+client-ids+consistently+between+clients+and+brokers
> >
> > Thanks
> >
>
>
>
> Unless stated otherwise above:
> IBM United Kingdom Limited - Registered in England and Wales with number
> 741598.
> Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU
>


[GitHub] kafka pull request #3856: MINOR: Remove unused SecurityProtocol.TRACE

2017-09-14 Thread ijuma
GitHub user ijuma opened a pull request:

https://github.com/apache/kafka/pull/3856

MINOR: Remove unused SecurityProtocol.TRACE

It adds complexity for no benefit since we don't use
it anywhere.

Also removed a few unused imports, variables and
default parameters.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/ijuma/kafka remove-security-protocol-trace

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3856.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3856


commit 92583abba75954cf84efe58e2d28a5632143269a
Author: Ismael Juma 
Date:   2017-09-14T09:38:23Z

MINOR: Remove unused SecurityProtocol.TRACE

It adds complexity for no benefit since we don't use
it anywhere.

commit c69e3be11afdda12b00993e3242e786ec33aac73
Author: Ismael Juma 
Date:   2017-09-14T09:39:00Z

Remove unused imports, variables and default parameters




---


[GitHub] kafka pull request #3854: MINOR: Fix LogContext message format in KafkaProdu...

2017-09-14 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3854


---


[GitHub] kafka pull request #3852: MINOR: Update TransactionManager to use LogContext

2017-09-14 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3852


---


[GitHub] kafka pull request #3857: MINOR: Bump version in streams quickstart archetyp...

2017-09-14 Thread dguy
GitHub user dguy opened a pull request:

https://github.com/apache/kafka/pull/3857

MINOR: Bump version in streams quickstart archetype pom.xml



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/dguy/kafka fix-archetype-version

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3857.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3857


commit 0566abbba4e8149074d38819f3c75736ead9585f
Author: Damian Guy 
Date:   2017-09-14T10:42:23Z

update version in streams archetype pom




---


[GitHub] kafka pull request #3858: MINOR: update docs to add note about removing SNAP...

2017-09-14 Thread dguy
GitHub user dguy opened a pull request:

https://github.com/apache/kafka/pull/3858

MINOR: update docs to add note about removing SNAPSHOT from streams 
dependency



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/dguy/kafka docs

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3858.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3858


commit e912689afe6fd2687934aa0b63176da28bec730c
Author: Damian Guy 
Date:   2017-09-14T10:50:10Z

update docs to remove SNAPSHOT




---


[GitHub] kafka pull request #3857: MINOR: Bump version in streams quickstart archetyp...

2017-09-14 Thread dguy
Github user dguy closed the pull request at:

https://github.com/apache/kafka/pull/3857


---


[jira] [Created] (KAFKA-5890) records.lag should use tags for topic and partition rather than using metric name.

2017-09-14 Thread Charly Molter (JIRA)
Charly Molter created KAFKA-5890:


 Summary: records.lag should use tags for topic and partition 
rather than using metric name.
 Key: KAFKA-5890
 URL: https://issues.apache.org/jira/browse/KAFKA-5890
 Project: Kafka
  Issue Type: Bug
  Components: clients
Affects Versions: 0.10.2.0
Reporter: Charly Molter


As part of KIP-92[1] a per partition lag metric was added.

These metrics are really useful, however in the implementation  it was 
implemented as a prefix to the metric name: 
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java#L1321-L1344

Usually these kind of metrics use tags and the name is constant for all topics, 
partitions.

We have a custom reporter which aggregates topics/partitions together to avoid 
explosion of the number of KPIs and this KPI doesn't support this as it doesn't 
have tags but a complex name.

[1] 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-92+-+Add+per+partition+lag+metrics+to+KafkaConsumer



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field

2017-09-14 Thread Artem Plotnikov (JIRA)
Artem Plotnikov created KAFKA-5891:
--

 Summary: Cast transformation fails if record schema contains 
timestamp field
 Key: KAFKA-5891
 URL: https://issues.apache.org/jira/browse/KAFKA-5891
 Project: Kafka
  Issue Type: Bug
  Components: KafkaConnect
Affects Versions: 0.11.0.0
Reporter: Artem Plotnikov


I have the following simple type cast transformation:
```
name=postgresql-source-simple
connector.class=io.confluent.connect.jdbc.JdbcSourceConnector
tasks.max=1

connection.url=jdbc:postgresql://localhost:5432/testdb?user=postgres&password=mysecretpassword
query=SELECT 1::INT as a, '2017-09-14 10:23:54'::TIMESTAMP as b

transforms=Cast
transforms.Cast.type=org.apache.kafka.connect.transforms.Cast$Value
transforms.Cast.spec=a:boolean

mode=bulk
topic.prefix=clients
```
Which fails with the following exception in runtime:
```
[2017-09-14 16:51:01,885] ERROR Task postgresql-source-simple-0 threw an 
uncaught and unrecoverable exception 
(org.apache.kafka.connect.runtime.WorkerTask:148)
org.apache.kafka.connect.errors.DataException: Invalid Java object for schema 
type INT64: class java.sql.Timestamp for field: "null"
at 
org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:239)
at 
org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:209)
at org.apache.kafka.connect.data.Struct.put(Struct.java:214)
at 
org.apache.kafka.connect.transforms.Cast.applyWithSchema(Cast.java:152)
at org.apache.kafka.connect.transforms.Cast.apply(Cast.java:108)
at 
org.apache.kafka.connect.runtime.TransformationChain.apply(TransformationChain.java:38)
at 
org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:190)
at 
org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:168)
at 
org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:146)
at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:190)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
  at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```
If I remove the  transforms.* part of the connector it will work correctly. 
Actually, it doesn't really matter which types I use in the transformation for 
field 'a', just the existence of a timestamp field brings the exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-14 Thread Bill Bejeck
+1 for me on collapsing the Rich and ValueWithKey interfaces into 1
interface.

Thanks,
Bill

On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov 
wrote:

> Hi Damian,
>
> Thanks for your feedback. Actually, this (what you propose) was the first
> idea of KIP-149. Then we decided to divide it into two KIPs. I also
> expressed my opinion that keeping the two interfaces (Rich and withKey)
> separate would add more overloads. So, email discussion resulted that this
> would not be a problem.
>
> Our initial idea was similar to :
>
> public abstract class RichValueMapper  implements
> ValueMapperWithKey, RichFunction {
> ..
> }
>
>
> So, we check the type of object, whether it is RichXXX or XXXWithKey inside
> the called method and continue accordingly.
>
> If this is ok with the community, I would like to revert the current design
> to this again.
>
> Cheers,
> Jeyhun
>
> On Wed, Sep 13, 2017 at 3:02 PM Damian Guy  wrote:
>
> > Hi Jeyhun,
> >
> > Thanks for sending out the update. I guess i was thinking more along the
> > lines of option 2 where we collapse the Rich and ValueWithKey etc
> > interfaces into 1 interface that has all of the arguments. I think we
> then
> > only need to add one additional overload for each operator?
> >
> > Thanks,
> > Damian
> >
> > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov 
> wrote:
> >
> > > Dear all,
> > >
> > > I would like to resume the discussion on KIP-159. I (and Guozhang)
> think
> > > that releasing KIP-149 and KIP-159 in the same release would make sense
> > to
> > > avoid a release with "partial" public APIs. There is a KIP [1] proposed
> > by
> > > Guozhang (and approved by me) to unify both KIPs.
> > > Please feel free to comment on this.
> > >
> > > [1]
> > >
> > https://cwiki.apache.org/confluence/pages/viewpage.
> action?pageId=73637757
> > >
> > > Cheers,
> > > Jeyhun
> > >
> > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov 
> > > wrote:
> > >
> > > > Hi Matthias, Damian, all,
> > > >
> > > > Thanks for your comments and sorry for super-late update.
> > > >
> > > > Sure, the DSL refactoring is not blocking for this KIP.
> > > > I made some changes to KIP document based on my prototype.
> > > >
> > > > Please feel free to comment.
> > > >
> > > > Cheers,
> > > > Jeyhun
> > > >
> > > > On Fri, Jul 7, 2017 at 9:35 PM Matthias J. Sax <
> matth...@confluent.io>
> > > > wrote:
> > > >
> > > >> I would not block this KIP with regard to DSL refactoring. IMHO, we
> > can
> > > >> just finish this one and the DSL refactoring will help later on to
> > > >> reduce the number of overloads.
> > > >>
> > > >> -Matthias
> > > >>
> > > >> On 7/7/17 5:28 AM, Jeyhun Karimov wrote:
> > > >> > I am following the related thread in the mailing list and looking
> > > >> forward
> > > >> > for one-shot solution for overloads issue.
> > > >> >
> > > >> > Cheers,
> > > >> > Jeyhun
> > > >> >
> > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy 
> > > >> wrote:
> > > >> >
> > > >> >> Hi Jeyhun,
> > > >> >>
> > > >> >> About overrides, what other alternatives do we have? For
> > > >> >>> backwards-compatibility we have to add extra methods to the
> > existing
> > > >> >> ones.
> > > >> >>>
> > > >> >>>
> > > >> >> It wasn't clear to me in the KIP if these are new methods or
> > > replacing
> > > >> >> existing ones.
> > > >> >> Also, we are currently discussing options for replacing the
> > > overrides.
> > > >> >>
> > > >> >> Thanks,
> > > >> >> Damian
> > > >> >>
> > > >> >>
> > > >> >>> About ProcessorContext vs RecordContext, you are right. I think
> I
> > > >> need to
> > > >> >>> implement a prototype to understand the full picture as some
> parts
> > > of
> > > >> the
> > > >> >>> KIP might not be as straightforward as I thought.
> > > >> >>>
> > > >> >>>
> > > >> >>> Cheers,
> > > >> >>> Jeyhun
> > > >> >>>
> > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> damian@gmail.com>
> > > >> wrote:
> > > >> >>>
> > > >>  HI Jeyhun,
> > > >> 
> > > >>  Is the intention that these methods are new overloads on the
> > > KStream,
> > > >>  KTable, etc?
> > > >> 
> > > >>  It is worth noting that a ProcessorContext is not a
> > RecordContext.
> > > A
> > > >>  RecordContext, as it stands, only exists during the processing
> > of a
> > > >> >>> single
> > > >>  record. Whereas the ProcessorContext exists for the lifetime of
> > the
> > > >>  Processor. Sot it doesn't make sense to cast a ProcessorContext
> > to
> > > a
> > > >>  RecordContext.
> > > >>  You mentioned above passing the InternalProcessorContext to the
> > > >> init()
> > > >>  calls. It is internal for a reason and i think it should remain
> > > that
> > > >> >> way.
> > > >>  It might be better to move the recordContext() method from
> > > >>  InternalProcessorContext to ProcessorContext.
> > > >> 
> > > >>  In the KIP you have an example showing:
> > > >>  richMapper.init((RecordContext) processorContext);
> > > >>  But the 

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-14 Thread Ted Yu
+1

One interface is cleaner.

On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck  wrote:

> +1 for me on collapsing the Rich and ValueWithKey interfaces into 1
> interface.
>
> Thanks,
> Bill
>
> On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov 
> wrote:
>
> > Hi Damian,
> >
> > Thanks for your feedback. Actually, this (what you propose) was the first
> > idea of KIP-149. Then we decided to divide it into two KIPs. I also
> > expressed my opinion that keeping the two interfaces (Rich and withKey)
> > separate would add more overloads. So, email discussion resulted that
> this
> > would not be a problem.
> >
> > Our initial idea was similar to :
> >
> > public abstract class RichValueMapper  implements
> > ValueMapperWithKey, RichFunction {
> > ..
> > }
> >
> >
> > So, we check the type of object, whether it is RichXXX or XXXWithKey
> inside
> > the called method and continue accordingly.
> >
> > If this is ok with the community, I would like to revert the current
> design
> > to this again.
> >
> > Cheers,
> > Jeyhun
> >
> > On Wed, Sep 13, 2017 at 3:02 PM Damian Guy  wrote:
> >
> > > Hi Jeyhun,
> > >
> > > Thanks for sending out the update. I guess i was thinking more along
> the
> > > lines of option 2 where we collapse the Rich and ValueWithKey
> etc
> > > interfaces into 1 interface that has all of the arguments. I think we
> > then
> > > only need to add one additional overload for each operator?
> > >
> > > Thanks,
> > > Damian
> > >
> > > On Wed, 13 Sep 2017 at 10:59 Jeyhun Karimov 
> > wrote:
> > >
> > > > Dear all,
> > > >
> > > > I would like to resume the discussion on KIP-159. I (and Guozhang)
> > think
> > > > that releasing KIP-149 and KIP-159 in the same release would make
> sense
> > > to
> > > > avoid a release with "partial" public APIs. There is a KIP [1]
> proposed
> > > by
> > > > Guozhang (and approved by me) to unify both KIPs.
> > > > Please feel free to comment on this.
> > > >
> > > > [1]
> > > >
> > > https://cwiki.apache.org/confluence/pages/viewpage.
> > action?pageId=73637757
> > > >
> > > > Cheers,
> > > > Jeyhun
> > > >
> > > > On Fri, Jul 21, 2017 at 2:00 AM Jeyhun Karimov  >
> > > > wrote:
> > > >
> > > > > Hi Matthias, Damian, all,
> > > > >
> > > > > Thanks for your comments and sorry for super-late update.
> > > > >
> > > > > Sure, the DSL refactoring is not blocking for this KIP.
> > > > > I made some changes to KIP document based on my prototype.
> > > > >
> > > > > Please feel free to comment.
> > > > >
> > > > > Cheers,
> > > > > Jeyhun
> > > > >
> > > > > On Fri, Jul 7, 2017 at 9:35 PM Matthias J. Sax <
> > matth...@confluent.io>
> > > > > wrote:
> > > > >
> > > > >> I would not block this KIP with regard to DSL refactoring. IMHO,
> we
> > > can
> > > > >> just finish this one and the DSL refactoring will help later on to
> > > > >> reduce the number of overloads.
> > > > >>
> > > > >> -Matthias
> > > > >>
> > > > >> On 7/7/17 5:28 AM, Jeyhun Karimov wrote:
> > > > >> > I am following the related thread in the mailing list and
> looking
> > > > >> forward
> > > > >> > for one-shot solution for overloads issue.
> > > > >> >
> > > > >> > Cheers,
> > > > >> > Jeyhun
> > > > >> >
> > > > >> > On Fri, Jul 7, 2017 at 10:32 AM Damian Guy <
> damian@gmail.com>
> > > > >> wrote:
> > > > >> >
> > > > >> >> Hi Jeyhun,
> > > > >> >>
> > > > >> >> About overrides, what other alternatives do we have? For
> > > > >> >>> backwards-compatibility we have to add extra methods to the
> > > existing
> > > > >> >> ones.
> > > > >> >>>
> > > > >> >>>
> > > > >> >> It wasn't clear to me in the KIP if these are new methods or
> > > > replacing
> > > > >> >> existing ones.
> > > > >> >> Also, we are currently discussing options for replacing the
> > > > overrides.
> > > > >> >>
> > > > >> >> Thanks,
> > > > >> >> Damian
> > > > >> >>
> > > > >> >>
> > > > >> >>> About ProcessorContext vs RecordContext, you are right. I
> think
> > I
> > > > >> need to
> > > > >> >>> implement a prototype to understand the full picture as some
> > parts
> > > > of
> > > > >> the
> > > > >> >>> KIP might not be as straightforward as I thought.
> > > > >> >>>
> > > > >> >>>
> > > > >> >>> Cheers,
> > > > >> >>> Jeyhun
> > > > >> >>>
> > > > >> >>> On Wed, Jul 5, 2017 at 10:40 AM Damian Guy <
> > damian@gmail.com>
> > > > >> wrote:
> > > > >> >>>
> > > > >>  HI Jeyhun,
> > > > >> 
> > > > >>  Is the intention that these methods are new overloads on the
> > > > KStream,
> > > > >>  KTable, etc?
> > > > >> 
> > > > >>  It is worth noting that a ProcessorContext is not a
> > > RecordContext.
> > > > A
> > > > >>  RecordContext, as it stands, only exists during the
> processing
> > > of a
> > > > >> >>> single
> > > > >>  record. Whereas the ProcessorContext exists for the lifetime
> of
> > > the
> > > > >>  Processor. Sot it doesn't make sense to cast a
> ProcessorContext
> > > to
> > > > a
> > > > >>  RecordContext.
> > > > >>  You mentioned above

[jira] [Created] (KAFKA-5892) Connector property override does not work unless setting ALL converter properties

2017-09-14 Thread Yeva Byzek (JIRA)
Yeva Byzek created KAFKA-5892:
-

 Summary: Connector property override does not work unless setting 
ALL converter properties
 Key: KAFKA-5892
 URL: https://issues.apache.org/jira/browse/KAFKA-5892
 Project: Kafka
  Issue Type: Improvement
  Components: KafkaConnect
Reporter: Yeva Byzek
Priority: Minor


A single connector setting override {{  "value.converter.schemas.enable": false 
}} won't take effect if not ALL the converter properties are overriden in the 
connector.

At minimum, we should give user warning or error that this is will be ignored.

We should also consider changing the behavior to allow the single property 
override even if all the converter properties are not specified, but this 
requires discussion to evaluate the impact of this change.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


QA bot has stalled

2017-09-14 Thread Ted Yu
Hi,
I logged INFRA-15084 .

FYI


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

2017-09-14 Thread Apache Jenkins Server
See 




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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[rajinisivaram] KAFKA-5783; Add KafkaPrincipalBuilder with support for SASL 
(KIP-189)

[ismael] MINOR: Fix LogContext message format in KafkaProducer

[ismael] MINOR: Update TransactionManager to use LogContext

--
[...truncated 2.54 MB...]

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
testNoMessagesSentExceptionFromOverlappingPatterns STARTED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
testNoMessagesSentExceptionFromOverlappingPatterns PASSED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
shouldAddStateStoreToRegexDefinedSource STARTED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
shouldAddStateStoreToRegexDefinedSource PASSED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithZeroSizedCache STARTED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithZeroSizedCache PASSED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithNonZeroSizedCache STARTED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithNonZeroSizedCache PASSED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldCompactTopicsForStateChangelogs STARTED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldCompactTopicsForStateChangelogs PASSED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldUseCompactAndDeleteForWindowStoreChangelogs STARTED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldUseCompactAndDeleteForWindowStoreChangelogs PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceSessionWindows STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceSessionWindows PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduce STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduce PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregate STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregate PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCount STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCount PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldGroupByKey STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldGroupByKey PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountWithInternalStore STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountWithInternalStore PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceWindowed STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceWindowed PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountSessionWindows STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountSessionWindows PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregateWindowed STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregateWindowed PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin PASSED

org.apache.kafka.streams.integration.ResetIntegrationTest > 
testReprocessingFromScratchAfterResetWithIntermediateUserTopic STARTED

org.apache.kafka.streams.integration.ResetIntegrationTest > 
testReprocessingFromScratchAfterResetWithIntermediateUserTopic FAILED
java.lang.AssertionError: expected:<0> but was:<1>
at org.junit.Assert.fail(Assert.java:88)
at org.junit.Assert.failNotEquals(Assert.java:834)
at org.junit.Assert.assertEquals(Assert.java:645)
at org.junit.Assert.assertEquals(Assert.java:631)
at 
org.apache.kafka.streams.integration.ResetIntegrationTest.cleanGlobal(ResetIntegrationTest.java:363)
at 
org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterResetWithIntermediateUserTopic(ResetIntegration

Build failed in Jenkins: kafka-trunk-jdk9 #9

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[rajinisivaram] KAFKA-5783; Add KafkaPrincipalBuilder with support for SASL 
(KIP-189)

[ismael] MINOR: Fix LogContext message format in KafkaProducer

[ismael] MINOR: Update TransactionManager to use LogContext

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on qnode3 (ubuntu) in workspace 

 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url https://github.com/apache/kafka.git # timeout=10
Fetching upstream changes from https://github.com/apache/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress https://github.com/apache/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
 > git rev-parse refs/remotes/origin/trunk^{commit} # timeout=10
 > git rev-parse refs/remotes/origin/origin/trunk^{commit} # timeout=10
Checking out Revision 2656659e0d7c0e427768ce216df2698acc8c9b11 
(refs/remotes/origin/trunk)
Commit message: "MINOR: Update TransactionManager to use LogContext"
 > git config core.sparsecheckout # timeout=10
 > git checkout -f 2656659e0d7c0e427768ce216df2698acc8c9b11
 > git rev-list f7b1add686c27bcf26519c8a6963d9a773d7ffbe # timeout=10
FATAL: Unable to produce a script file
java.io.IOException: No space left on device
at java.io.FileOutputStream.writeBytes(Native Method)
at java.io.FileOutputStream.write(FileOutputStream.java:326)
at sun.nio.cs.StreamEncoder.writeBytes(StreamEncoder.java:221)
at sun.nio.cs.StreamEncoder.implClose(StreamEncoder.java:316)
at sun.nio.cs.StreamEncoder.close(StreamEncoder.java:149)
at java.io.OutputStreamWriter.close(OutputStreamWriter.java:233)
at hudson.FilePath$17.invoke(FilePath.java:1380)
at hudson.FilePath$17.invoke(FilePath.java:1363)
at hudson.FilePath$FileCallableWrapper.call(FilePath.java:2739)
at hudson.remoting.UserRequest.perform(UserRequest.java:153)
at hudson.remoting.UserRequest.perform(UserRequest.java:50)
at hudson.remoting.Request$2.run(Request.java:336)
at 
hudson.remoting.InterceptingExecutorService$1.call(InterceptingExecutorService.java:68)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
at ..remote call to qnode3(Native Method)
at hudson.remoting.Channel.attachCallSiteStackTrace(Channel.java:1545)
at hudson.remoting.UserResponse.retrieve(UserRequest.java:253)
at hudson.remoting.Channel.call(Channel.java:830)
at hudson.FilePath.act(FilePath.java:986)
Caused: java.io.IOException: remote file operation failed: 
 at 
hudson.remoting.Channel@23b449a:qnode3
at hudson.FilePath.act(FilePath.java:993)
at hudson.FilePath.act(FilePath.java:975)
at hudson.FilePath.createTextTempFile(FilePath.java:1363)
Caused: java.io.IOException: Failed to create a temp file on 

at hudson.FilePath.createTextTempFile(FilePath.java:1386)
at 
hudson.tasks.CommandInterpreter.createScriptFile(CommandInterpreter.java:162)
at hudson.tasks.CommandInterpreter.perform(CommandInterpreter.java:94)
at hudson.tasks.CommandInterpreter.perform(CommandInterpreter.java:66)
at hudson.tasks.BuildStepMonitor$1.perform(BuildStepMonitor.java:20)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.perform(AbstractBuild.java:735)
at hudson.model.Build$BuildExecution.build(Build.java:206)
at hudson.model.Build$BuildExecution.doRun(Build.java:163)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.run(AbstractBuild.java:490)
at hudson.model.Run.execute(Run.java:1735)
at hudson.model.FreeStyleBuild.run(FreeStyleBuild.java:43)
at hudson.model.ResourceController.execute(ResourceController.java:97)
at hudson.model.Executor.run(Executor.java:405)
Build step 'Execute shell' marked build as failure
Recording test results
Setting 
GRADLE_3_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_3.4-rc-2
ERROR: Step ‘Publish JUnit test result report’ failed: Test reports were found 
but none of them are new. Did tests run? 
For example, 

 is 19 hr old

Setting 
GRADLE_3_4_RC_2_HOME=/home/jenkins/jenkins-slave/tools/hudson.plugins.gradle.GradleInstallation/Gradle_3.4-rc-2
Not sending mail 

[jira] [Created] (KAFKA-5893) ResetIntegrationTest fails

2017-09-14 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-5893:
--

 Summary: ResetIntegrationTest fails
 Key: KAFKA-5893
 URL: https://issues.apache.org/jira/browse/KAFKA-5893
 Project: Kafka
  Issue Type: Bug
  Components: streams, unit tests
Affects Versions: 0.11.0.0, 0.11.0.1
Reporter: Matthias J. Sax
Assignee: Matthias J. Sax


{noformat}
java.lang.AssertionError: expected:<0> but was:<1>
at org.junit.Assert.fail(Assert.java:88)
at org.junit.Assert.failNotEquals(Assert.java:834)
at org.junit.Assert.assertEquals(Assert.java:645)
at org.junit.Assert.assertEquals(Assert.java:631)
at 
org.apache.kafka.streams.integration.ResetIntegrationTest.cleanGlobal(ResetIntegrationTest.java:363)
at 
org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterResetWithIntermediateUserTopic(ResetIntegrationTest.java:172)
{noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] kafka pull request #3859: KAFKA-5893: ResetIntegrationTest fails

2017-09-14 Thread mjsax
GitHub user mjsax opened a pull request:

https://github.com/apache/kafka/pull/3859

KAFKA-5893: ResetIntegrationTest fails

 - improve stderr output for better debugging

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/mjsax/kafka kafka-5893-reset-integration-test

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3859.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3859


commit fa51ab01cf0227172fe9cef5c1b247fff303569f
Author: Matthias J. Sax 
Date:   2017-09-14T17:11:23Z

KAFKA-5893: ResetIntegrationTest fails
 - improve stderr output for better debugging




---


Re: [VOTE] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Jason Gustafson
+1. Thanks for the KIP.

On Thu, Sep 14, 2017 at 2:33 AM, Rajini Sivaram 
wrote:

> +1 (binding)
>
> Thank you for the KIP, Mickael.
>
> On Thu, Sep 14, 2017 at 10:24 AM, Edoardo Comar  wrote:
>
> > +1 (non-binding)
> > --
> >
> > Edoardo Comar
> >
> > IBM Message Hub
> >
> > IBM UK Ltd, Hursley Park, SO21 2JN
> >
> >
> >
> > From:   Ismael Juma 
> > To: dev@kafka.apache.org
> > Date:   14/09/2017 09:56
> > Subject:Re: [VOTE] KIP-190: Handle client-ids consistently
> between
> > clients and brokers
> > Sent by:isma...@gmail.com
> >
> >
> >
> > Thanks for the KIP. +1 (binding) from me.
> >
> > Because this is a bug fix, the change is simple and the impact can be
> bad,
> > I suggest considering this for the 1.0.0 release. Guozhang's call though.
> >
> > Ismael
> >
> > On Thu, Sep 14, 2017 at 9:52 AM, Mickael Maison <
> mickael.mai...@gmail.com>
> > wrote:
> >
> > > Hi,
> > >
> > > I'd like to start the vote on KIP-190:
> > >
> > https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.
> > apache.org_confluence_display_KAFKA_KIP-2D&d=DwIBaQ&c=jf_
> > iaSHvJObTbx-siA1ZOg&r=EzRhmSah4IHsUZVekRUIINhltZK7U0
> > OaeRo7hgW4_tQ&m=6-i-zyk332E_rCIPJN_-IvBSuVdiMjzXzl0BUQFQmWU&s=
> > WUK0rNOyKKGFJFCQ5DF0VnfaceASTLdRZSwqDXK3x3Q&e=
> >
> > > 190%3A+Handle+client-ids+consistently+between+clients+and+brokers
> > >
> > > Thanks
> > >
> >
> >
> >
> > Unless stated otherwise above:
> > IBM United Kingdom Limited - Registered in England and Wales with number
> > 741598.
> > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> 3AU
> >
>


[jira] [Created] (KAFKA-5894) add the notion of max inflight requests to async ZookeeperClient

2017-09-14 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-5894:
---

 Summary: add the notion of max inflight requests to async 
ZookeeperClient
 Key: KAFKA-5894
 URL: https://issues.apache.org/jira/browse/KAFKA-5894
 Project: Kafka
  Issue Type: Sub-task
Reporter: Onur Karaman
Assignee: Onur Karaman


ZookeeperClient is a zookeeper client that encourages pipelined requests to 
zookeeper. We want to add the notion of max inflight requests to the client for 
several reasons:
# to bound memory overhead associated with async requests on the client.
# to not overwhelm the zookeeper ensemble with a burst of requests.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] kafka pull request #3860: KAFKA-5894: add the notion of max inflight request...

2017-09-14 Thread onurkaraman
GitHub user onurkaraman opened a pull request:

https://github.com/apache/kafka/pull/3860

KAFKA-5894: add the notion of max inflight requests to async 
ZookeepeeperClient

ZookeeperClient is a zookeeper client that encourages pipelined requests to 
zookeeper. We want to add the notion of max inflight requests to the client for 
several reasons:
1. to bound memory overhead associated with async requests on the client.
2. to not overwhelm the zookeeper ensemble with a burst of requests.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/onurkaraman/kafka KAFKA-5894

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3860.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3860


commit a767c27436d4dc4da452edcee8bff54edd41dabd
Author: Onur Karaman 
Date:   2017-09-14T17:46:22Z

KAFKA-5894: add the notion of max inflight requests to async ZookeeperClient

ZookeeperClient is a zookeeper client that encourages pipelined requests to 
zookeeper. We want to add the notion of max inflight requests to the client for 
several reasons:
1. to bound memory overhead associated with async requests on the client.
2. to not overwhelm the zookeeper ensemble with a burst of requests.




---


Re: [VOTE] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Guozhang Wang
This KIP has three binding votes already but we need to wait 72 hours for
the voting thread to complete. I'd take it as "tentative accepted" for
1.0.0 unless someone has a -1 vote before the voting is closed.


Guozhang

On Thu, Sep 14, 2017 at 10:28 AM, Jason Gustafson 
wrote:

> +1. Thanks for the KIP.
>
> On Thu, Sep 14, 2017 at 2:33 AM, Rajini Sivaram 
> wrote:
>
> > +1 (binding)
> >
> > Thank you for the KIP, Mickael.
> >
> > On Thu, Sep 14, 2017 at 10:24 AM, Edoardo Comar 
> wrote:
> >
> > > +1 (non-binding)
> > > --
> > >
> > > Edoardo Comar
> > >
> > > IBM Message Hub
> > >
> > > IBM UK Ltd, Hursley Park, SO21 2JN
> > >
> > >
> > >
> > > From:   Ismael Juma 
> > > To: dev@kafka.apache.org
> > > Date:   14/09/2017 09:56
> > > Subject:Re: [VOTE] KIP-190: Handle client-ids consistently
> > between
> > > clients and brokers
> > > Sent by:isma...@gmail.com
> > >
> > >
> > >
> > > Thanks for the KIP. +1 (binding) from me.
> > >
> > > Because this is a bug fix, the change is simple and the impact can be
> > bad,
> > > I suggest considering this for the 1.0.0 release. Guozhang's call
> though.
> > >
> > > Ismael
> > >
> > > On Thu, Sep 14, 2017 at 9:52 AM, Mickael Maison <
> > mickael.mai...@gmail.com>
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > I'd like to start the vote on KIP-190:
> > > >
> > > https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.
> > > apache.org_confluence_display_KAFKA_KIP-2D&d=DwIBaQ&c=jf_
> > > iaSHvJObTbx-siA1ZOg&r=EzRhmSah4IHsUZVekRUIINhltZK7U0
> > > OaeRo7hgW4_tQ&m=6-i-zyk332E_rCIPJN_-IvBSuVdiMjzXzl0BUQFQmWU&s=
> > > WUK0rNOyKKGFJFCQ5DF0VnfaceASTLdRZSwqDXK3x3Q&e=
> > >
> > > > 190%3A+Handle+client-ids+consistently+between+clients+and+brokers
> > > >
> > > > Thanks
> > > >
> > >
> > >
> > >
> > > Unless stated otherwise above:
> > > IBM United Kingdom Limited - Registered in England and Wales with
> number
> > > 741598.
> > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> > 3AU
> > >
> >
>



-- 
-- Guozhang


[jira] [Created] (KAFKA-5895) Update readme to reflect that Gradle 2 is no longer good enough

2017-09-14 Thread JIRA
Matthias Weßendorf created KAFKA-5895:
-

 Summary: Update readme to reflect that Gradle 2 is no longer good 
enough
 Key: KAFKA-5895
 URL: https://issues.apache.org/jira/browse/KAFKA-5895
 Project: Kafka
  Issue Type: Improvement
  Components: documentation
Affects Versions: 0.11.0.2
Reporter: Matthias Weßendorf
Priority: Trivial


The README says:

Kafka requires Gradle 2.0 or higher.

but running with "2.13", I am getting an ERROR message, saying that 3.0+ is 
needed:

{code}
> Failed to apply plugin [class 
> 'com.github.jengelman.gradle.plugins.shadow.ShadowBasePlugin']
   > This version of Shadow supports Gradle 3.0+ only. Please upgrade.

{code}

Full log here:

{code}
➜  kafka git:(utils_improvment) ✗ gradle 
To honour the JVM settings for this build a new JVM will be forked. Please 
consider using the daemon: 
https://docs.gradle.org/2.13/userguide/gradle_daemon.html.
Download https://jcenter.bintray.com/org/ajoberstar/grgit/1.9.3/grgit-1.9.3.pom
Download 
https://jcenter.bintray.com/com/github/ben-manes/gradle-versions-plugin/0.15.0/gradle-versions-plugin-0.15.0.pom
Download 
https://repo1.maven.org/maven2/org/scoverage/gradle-scoverage/2.1.0/gradle-scoverage-2.1.0.pom
Download 
https://jcenter.bintray.com/com/github/jengelman/gradle/plugins/shadow/2.0.1/shadow-2.0.1.pom
Download 
https://repo1.maven.org/maven2/org/eclipse/jgit/org.eclipse.jgit/4.5.2.201704071617-r/org.eclipse.jgit-4.5.2.201704071617-r.pom
Download 
https://repo1.maven.org/maven2/org/eclipse/jgit/org.eclipse.jgit-parent/4.5.2.201704071617-r/org.eclipse.jgit-parent-4.5.2.201704071617-r.pom
Download 
https://repo1.maven.org/maven2/org/eclipse/jgit/org.eclipse.jgit.ui/4.5.2.201704071617-r/org.eclipse.jgit.ui-4.5.2.201704071617-r.pom
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.jsch/0.0.9/jsch.agentproxy.jsch-0.0.9.pom
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy/0.0.9/jsch.agentproxy-0.0.9.pom
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.pageant/0.0.9/jsch.agentproxy.pageant-0.0.9.pom
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.sshagent/0.0.9/jsch.agentproxy.sshagent-0.0.9.pom
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.usocket-jna/0.0.9/jsch.agentproxy.usocket-jna-0.0.9.pom
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.usocket-nc/0.0.9/jsch.agentproxy.usocket-nc-0.0.9.pom
Download https://repo1.maven.org/maven2/com/jcraft/jsch/0.1.54/jsch-0.1.54.pom
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.core/0.0.9/jsch.agentproxy.core-0.0.9.pom
Download https://jcenter.bintray.com/org/ajoberstar/grgit/1.9.3/grgit-1.9.3.jar
Download 
https://jcenter.bintray.com/com/github/ben-manes/gradle-versions-plugin/0.15.0/gradle-versions-plugin-0.15.0.jar
Download 
https://repo1.maven.org/maven2/org/scoverage/gradle-scoverage/2.1.0/gradle-scoverage-2.1.0.jar
Download 
https://jcenter.bintray.com/com/github/jengelman/gradle/plugins/shadow/2.0.1/shadow-2.0.1.jar
Download 
https://repo1.maven.org/maven2/org/eclipse/jgit/org.eclipse.jgit/4.5.2.201704071617-r/org.eclipse.jgit-4.5.2.201704071617-r.jar
Download 
https://repo1.maven.org/maven2/org/eclipse/jgit/org.eclipse.jgit.ui/4.5.2.201704071617-r/org.eclipse.jgit.ui-4.5.2.201704071617-r.jar
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.jsch/0.0.9/jsch.agentproxy.jsch-0.0.9.jar
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.pageant/0.0.9/jsch.agentproxy.pageant-0.0.9.jar
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.sshagent/0.0.9/jsch.agentproxy.sshagent-0.0.9.jar
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.usocket-jna/0.0.9/jsch.agentproxy.usocket-jna-0.0.9.jar
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.usocket-nc/0.0.9/jsch.agentproxy.usocket-nc-0.0.9.jar
Download 
https://repo1.maven.org/maven2/com/jcraft/jsch.agentproxy.core/0.0.9/jsch.agentproxy.core-0.0.9.jar
Building project 'core' with Scala version 2.11.11

FAILURE: Build failed with an exception.

* Where:
Build file '/home/Matthias/Work/Apache/kafka/build.gradle' line: 978

* What went wrong:
A problem occurred evaluating root project 'kafka'.
> Failed to apply plugin [class 
> 'com.github.jengelman.gradle.plugins.shadow.ShadowBasePlugin']
   > This version of Shadow supports Gradle 3.0+ only. Please upgrade.

* Try:
Run with --stacktrace option to get the stack trace. Run with --info or --debug 
option to get more log output.

BUILD FAILED

Total time: 14.637 secs
➜  kafka git:(utils_improvment) ✗ gradle --version


Gradle 2.13
{code} 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [VOTE] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Gwen Shapira
+1 (binding)

On Thu, Sep 14, 2017 at 11:25 AM Guozhang Wang  wrote:

> This KIP has three binding votes already but we need to wait 72 hours for
> the voting thread to complete. I'd take it as "tentative accepted" for
> 1.0.0 unless someone has a -1 vote before the voting is closed.
>
>
> Guozhang
>
> On Thu, Sep 14, 2017 at 10:28 AM, Jason Gustafson 
> wrote:
>
> > +1. Thanks for the KIP.
> >
> > On Thu, Sep 14, 2017 at 2:33 AM, Rajini Sivaram  >
> > wrote:
> >
> > > +1 (binding)
> > >
> > > Thank you for the KIP, Mickael.
> > >
> > > On Thu, Sep 14, 2017 at 10:24 AM, Edoardo Comar 
> > wrote:
> > >
> > > > +1 (non-binding)
> > > > --
> > > >
> > > > Edoardo Comar
> > > >
> > > > IBM Message Hub
> > > >
> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> > > >
> > > >
> > > >
> > > > From:   Ismael Juma 
> > > > To: dev@kafka.apache.org
> > > > Date:   14/09/2017 09:56
> > > > Subject:Re: [VOTE] KIP-190: Handle client-ids consistently
> > > between
> > > > clients and brokers
> > > > Sent by:isma...@gmail.com
> > > >
> > > >
> > > >
> > > > Thanks for the KIP. +1 (binding) from me.
> > > >
> > > > Because this is a bug fix, the change is simple and the impact can be
> > > bad,
> > > > I suggest considering this for the 1.0.0 release. Guozhang's call
> > though.
> > > >
> > > > Ismael
> > > >
> > > > On Thu, Sep 14, 2017 at 9:52 AM, Mickael Maison <
> > > mickael.mai...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > I'd like to start the vote on KIP-190:
> > > > >
> > > > https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.
> > > > apache.org_confluence_display_KAFKA_KIP-2D&d=DwIBaQ&c=jf_
> > > > iaSHvJObTbx-siA1ZOg&r=EzRhmSah4IHsUZVekRUIINhltZK7U0
> > > > OaeRo7hgW4_tQ&m=6-i-zyk332E_rCIPJN_-IvBSuVdiMjzXzl0BUQFQmWU&s=
> > > > WUK0rNOyKKGFJFCQ5DF0VnfaceASTLdRZSwqDXK3x3Q&e=
> > > >
> > > > > 190%3A+Handle+client-ids+consistently+between+clients+and+brokers
> > > > >
> > > > > Thanks
> > > > >
> > > >
> > > >
> > > >
> > > > Unless stated otherwise above:
> > > > IBM United Kingdom Limited - Registered in England and Wales with
> > number
> > > > 741598.
> > > > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire
> PO6
> > > 3AU
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>


[GitHub] kafka pull request #3861: KAFKA-5895

2017-09-14 Thread matzew
GitHub user matzew opened a pull request:

https://github.com/apache/kafka/pull/3861

KAFKA-5895 

As discussed in 
[KAFKA-5895](https://issues.apache.org/jira/browse/KAFKA-5895), this trivial PR 
is simply adding a hint that Gradle 3.0+ is now needed

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/matzew/kafka Correct_Gradle_Info

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3861.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3861


commit 17ab1c6f0bdf883ef85efb5dba9bfa469f855688
Author: Matthias Wessendorf 
Date:   2017-09-14T18:38:33Z

Adding hint that Gradle 3.0+ is now needed




---


[jira] [Created] (KAFKA-5896) Kafka Connect task threads never interrupted

2017-09-14 Thread Nick Pillitteri (JIRA)
Nick Pillitteri created KAFKA-5896:
--

 Summary: Kafka Connect task threads never interrupted
 Key: KAFKA-5896
 URL: https://issues.apache.org/jira/browse/KAFKA-5896
 Project: Kafka
  Issue Type: Bug
  Components: KafkaConnect
Reporter: Nick Pillitteri
Priority: Minor


h2. Problem

Kafka Connect tasks associated with connectors are run in their own threads. 
When tasks are stopped or restarted, a flag is set - {{stopping}} - to indicate 
the task should stop processing records. However, if the thread the task is 
running in is blocked (waiting for a lock or performing I/O) it's possible the 
task will never stop.

I've created a connector specifically to demonstrate this issue (along with 
some more detailed instructions for reproducing the issue): 
https://github.com/smarter-travel-media/hang-connector

I believe this is an issue because it means that a single badly behaved 
connector (any connector that does I/O without timeouts) can cause the Kafka 
Connect worker to get into a state where the only solution is to restart the 
JVM.

I think, but couldn't reproduce, that this is the cause of this problem on 
Stack Overflow: 
https://stackoverflow.com/questions/43802156/inconsistent-connector-state-connectexception-task-already-exists-in-this-work

h2. Expected Result

I would expect the Worker to eventually interrupt the thread that the task is 
running in. In the past across various other libraries, this is what I've seen 
done when a thread needs to be forcibly stopped.

h2. Actual Result

In actuality, the Worker sets a {{stopping}} flag and lets the thread run 
indefinitely. It uses a timeout while waiting for the task to stop but after 
this timeout has expired it simply sets a {{cancelled}} flag. This means that 
every time a task is restarted, a new thread running the task will be created. 
Thus a task may end up with multiple instances all running in their own threads 
when there's only supposed to be a single thread.

h2. Steps to Reproduce

The problem can be replicated by using the connector available here: 
https://github.com/smarter-travel-media/hang-connector

Apologies for how involved the steps are.

I've created a patch that forcibly interrupts threads after they fail to 
gracefully shutdown here: 
https://github.com/smarter-travel-media/kafka/commit/295c747a9fd82ee8b30556c89c31e0bfcce5a2c5

I've confirmed that this fixes the issue. I can add some unit tests and submit 
a PR if people agree that this is a bug and interrupting threads is the right 
fix.

Thanks!



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


1.0.0 Feature Freeze Update

2017-09-14 Thread Guozhang Wang
Hello folks,

This is a heads up on 1.0.0 progress post the KIP-deadline:

https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=71764913


In this release we have a total of 33 KIPs either newly adopted or
inherited from the previous release. Among them 15 KIPs has been merged to
trunk, and others are still ongoing / under reviews.

Please note that the feature freeze deadline is Sept. 20th (next
Wednesday), and after that deadline any accepted KIPs will be pushed to the
next minor release. For those KIP proposer / guiding committer: if you are
confident that the KIP's PRs can be merged to trunk, please make sure to do
so before next Wednesday.


Thanks,
-- Guozhang


Re: [VOTE] KIP-182 - Reduce Streams DSL overloads and allow easier use of custom storage engines

2017-09-14 Thread Guozhang Wang
I'd suggest we remove both to and through together in KIP-182, since for
operator "KTable#to" is as confusing as to "KTable#through" which
overwhelms its benefit as a syntax sugar. I think the extra step "toStream"
is actually better to remind the caller that it is sending its changelog
stream to topic, plus it is not that much characters.


Guozhang

On Wed, Sep 13, 2017 at 12:40 AM, Damian Guy  wrote:

> Hi Guozhang,
>
> I had an offline discussion with Matthias and Bill about it. It is thought
> that `to` offers some benefit, i.e., syntactic sugar, so perhaps no harm in
> keeping it. However, `through` less so, seeing as we can materialize stores
> via `filter`, `map` etc, so one of the main benefits of `through` no longer
> exists. WDYT?
>
> Thanks,
> Damian
>
> On Tue, 12 Sep 2017 at 18:17 Guozhang Wang  wrote:
>
> > Hi Damian,
> >
> > Why we are deprecating KTable.through while keeping KTable.to? Should we
> > either keep both of them or deprecate both of them in favor or
> > KTable.toStream if people agree that it is confusing to users?
> >
> >
> > Guozhang
> >
> >
> > On Tue, Sep 12, 2017 at 1:18 AM, Damian Guy 
> wrote:
> >
> > > Hi All,
> > >
> > > A minor update to the KIP, i needed to add KTable.to(Produced) for
> > > consistency. KTable.through will be deprecated in favour of using
> > > KTable.toStream().through()
> > >
> > > Thanks,
> > > Damian
> > >
> > > On Thu, 7 Sep 2017 at 08:52 Damian Guy  wrote:
> > >
> > > > Thanks all. The vote is now closed and the KIP has been accepted
> with:
> > > > 2 non binding votes - bill and matthias
> > > > 3 binding  - Damian, Guozhang, Sriram
> > > >
> > > > Regards,
> > > > Damian
> > > >
> > > > On Tue, 5 Sep 2017 at 22:24 Sriram Subramanian 
> > wrote:
> > > >
> > > >> +1
> > > >>
> > > >> On Tue, Sep 5, 2017 at 1:33 PM, Guozhang Wang 
> > > wrote:
> > > >>
> > > >> > +1
> > > >> >
> > > >> > On Fri, Sep 1, 2017 at 3:45 PM, Matthias J. Sax <
> > > matth...@confluent.io>
> > > >> > wrote:
> > > >> >
> > > >> > > +1
> > > >> > >
> > > >> > > On 9/1/17 2:53 PM, Bill Bejeck wrote:
> > > >> > > > +1
> > > >> > > >
> > > >> > > > On Thu, Aug 31, 2017 at 10:20 AM, Damian Guy <
> > > damian@gmail.com>
> > > >> > > wrote:
> > > >> > > >
> > > >> > > >> Thanks everyone for voting! Unfortunately i've had to make a
> > bit
> > > >> of an
> > > >> > > >> update based on some issues found during implementation.
> > > >> > > >> The main changes are:
> > > >> > > >> BytesStoreSupplier -> StoreSupplier
> > > >> > > >> Addition of:
> > > >> > > >> WindowBytesStoreSupplier, KeyValueBytesStoreSupplier,
> > > >> > > >> SessionBytesStoreSupplier that will restrict store types to
> > >  > > >> > > byte[]>
> > > >> > > >> 3 new overloads added to Materialized to enable developers to
> > > >> create a
> > > >> > > >> Materialized of the appropriate type, i..e, WindowStore etc
> > > >> > > >> Update DSL where Materialized is used such that the stores
> have
> > > >> > generic
> > > >> > > >> types of 
> > > >> > > >> Some minor changes to the arguments to
> > > Store#persistentWindowStore
> > > >> and
> > > >> > > >> Store#persistentSessionStore
> > > >> > > >>
> > > >> > > >> Please take a look and recast the votes.
> > > >> > > >>
> > > >> > > >> Thanks for your time,
> > > >> > > >> Damian
> > > >> > > >>
> > > >> > > >> On Fri, 25 Aug 2017 at 17:05 Matthias J. Sax <
> > > >> matth...@confluent.io>
> > > >> > > >> wrote:
> > > >> > > >>
> > > >> > > >>> Thanks Damian. Great KIP!
> > > >> > > >>>
> > > >> > > >>> +1
> > > >> > > >>>
> > > >> > > >>>
> > > >> > > >>> -Matthias
> > > >> > > >>>
> > > >> > > >>> On 8/25/17 6:45 AM, Damian Guy wrote:
> > > >> > >  Hi,
> > > >> > > 
> > > >> > >  I've just realised we need to add two methods to
> > > >> StateStoreBuilder
> > > >> > or
> > > >> > > >> it
> > > >> > >  isn't going to work:
> > > >> > > 
> > > >> > >  Map logConfig();
> > > >> > >  boolean loggingEnabled();
> > > >> > > 
> > > >> > >  These are needed when we are building the topology and
> > > >> determining
> > > >> > >  changelog topic names and configs.
> > > >> > > 
> > > >> > > 
> > > >> > >  I've also update the KIP to add
> > > >> > > 
> > > >> > >  StreamBuilder#stream(String topic)
> > > >> > > 
> > > >> > >  StreamBuilder#stream(String topic, Consumed options)
> > > >> > > 
> > > >> > > 
> > > >> > >  Thanks
> > > >> > > 
> > > >> > > 
> > > >> > >  On Thu, 24 Aug 2017 at 22:11 Sriram Subramanian <
> > > >> r...@confluent.io>
> > > >> > > >>> wrote:
> > > >> > > 
> > > >> > > > +1
> > > >> > > >
> > > >> > > > On Thu, Aug 24, 2017 at 10:20 AM, Guozhang Wang <
> > > >> > wangg...@gmail.com>
> > > >> > > > wrote:
> > > >> > > >
> > > >> > > >> +1. Thanks Damian!
> > > >> > > >>
> > > >> > > >> On Thu, Aug 24, 2017 at 9:47 AM, Bill Bejeck <
> > > >> bbej...@gmail.com>
> > > >> > > >>> wrote:
> > > >> > > >>
> > > >>

[jira] [Created] (KAFKA-5897) The producerId can be reset unnecessarily

2017-09-14 Thread Apurva Mehta (JIRA)
Apurva Mehta created KAFKA-5897:
---

 Summary: The producerId can be reset unnecessarily
 Key: KAFKA-5897
 URL: https://issues.apache.org/jira/browse/KAFKA-5897
 Project: Kafka
  Issue Type: Improvement
Affects Versions: 0.11.0.0, 1.0.0
Reporter: Apurva Mehta


Currently, we expire batches and reset the producer id in cases where we don't 
need to.

For instance, if a batch which has been previously sent is expired in the 
accumulator, the producerId is reset (or the transaction aborted) 
unconditionally. However, if the batch failed with certain error codes like 
{{NOT_LEADER_FOR_PARTITION}}, etc., which definitively indicate that the write 
never succeeded, we don't need to reset the producer state since the status of 
the batch is not in doubt. 

Essentially, we would like an 'reset based on failure mode' logic which would 
be a bit smarter.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


Re: [VOTE] KIP-190: Handle client-ids consistently between clients and brokers

2017-09-14 Thread Jun Rao
Hi, Mickael,

Thanks for the KIP. +1 from me.

Jun

On Thu, Sep 14, 2017 at 1:52 AM, Mickael Maison 
wrote:

> Hi,
>
> I'd like to start the vote on KIP-190:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 190%3A+Handle+client-ids+consistently+between+clients+and+brokers
>
> Thanks
>


[GitHub] kafka pull request #3862: DOCS-90 - Add navigation to Streams doc

2017-09-14 Thread joel-hamill
GitHub user joel-hamill opened a pull request:

https://github.com/apache/kafka/pull/3862

DOCS-90 - Add navigation to Streams doc



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/joel-hamill/kafka 
joel-hamill/streams-dev-guide

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3862.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3862


commit e304c6d868bd985028df15385985e5044956b9a1
Author: Joel Hamill 
Date:   2017-09-14T19:13:40Z

DOCS-90 - Add navigation to Streams doc




---


[GitHub] kafka-site pull request #76: DOCS-90 - Add streams child topics to left-hand...

2017-09-14 Thread joel-hamill
GitHub user joel-hamill opened a pull request:

https://github.com/apache/kafka-site/pull/76

DOCS-90 - Add streams child topics to left-hand nav

https://confluentinc.atlassian.net/browse/DOCS-90

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/joel-hamill/kafka-site 
joel-hamill/nav-update-streams

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka-site/pull/76.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #76


commit 45b3713f9c8bb4afa543a0708a9bbe24fbc512f7
Author: Joel Hamill 
Date:   2017-09-14T19:25:36Z

DOCS-90 - Add streams child topics to left-hand nav




---


[GitHub] kafka-site pull request #76: DOCS-90 - Add streams child topics to left-hand...

2017-09-14 Thread joel-hamill
Github user joel-hamill closed the pull request at:

https://github.com/apache/kafka-site/pull/76


---


[GitHub] kafka-site pull request #77: DOCS-90 - Add streams child topics to left-hand...

2017-09-14 Thread joel-hamill
GitHub user joel-hamill opened a pull request:

https://github.com/apache/kafka-site/pull/77

DOCS-90 - Add streams child topics to left-hand nav

https://confluentinc.atlassian.net/browse/DOCS-90

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/joel-hamill/kafka-site 
joel-hamill/nav-fixes-streams

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka-site/pull/77.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #77


commit 0dec1d9f53ed3d16b1cccb531124e9ab0fed2be1
Author: Joel Hamill 
Date:   2017-09-14T19:36:19Z

DOCS-90 - Add streams child topics to left-hand nav




---


[GitHub] kafka pull request #3856: MINOR: Remove unused SecurityProtocol.TRACE

2017-09-14 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3856


---


[GitHub] kafka pull request #3863: MINOR: Use SecurityProtocol in AuthenticationConte...

2017-09-14 Thread hachikuji
GitHub user hachikuji opened a pull request:

https://github.com/apache/kafka/pull/3863

MINOR: Use SecurityProtocol in AuthenticationContext

Since we removed the unused `TRACE` option from `SecurityProtocol`, it now 
seems safer to expose it from `AuthenticationContext`. Additionally this patch 
exposes javadocs under security.auth and relocates the `Login` and 
`AuthCallbackHandler` to a non-public package.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/hachikuji/kafka 
use-security-protocol-in-auth-context

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3863.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3863


commit 7ea23a1e5f873ae9822ba4946c88ca3072c2674b
Author: Jason Gustafson 
Date:   2017-09-14T21:04:50Z

MINOR: Use SecurityProtocol in AuthenticationContext




---


[GitHub] kafka-site issue #77: DOCS-90 - Add streams child topics to left-hand nav

2017-09-14 Thread joel-hamill
Github user joel-hamill commented on the issue:

https://github.com/apache/kafka-site/pull/77
  
ping for review @guozhangwang @dguy 


---


Jenkins build is back to normal : kafka-trunk-jdk9 #10

2017-09-14 Thread Apache Jenkins Server
See 




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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[jason] MINOR: Remove unused SecurityProtocol.TRACE

--
[...truncated 931.17 KB...]
kafka.server.ServerGenerateBrokerIdTest > testAutoGenerateBrokerId PASSED

kafka.server.ServerGenerateBrokerIdTest > testMultipleLogDirsMetaProps STARTED

kafka.server.ServerGenerateBrokerIdTest > testMultipleLogDirsMetaProps PASSED

kafka.server.ServerGenerateBrokerIdTest > testDisableGeneratedBrokerId STARTED

kafka.server.ServerGenerateBrokerIdTest > testDisableGeneratedBrokerId PASSED

kafka.server.ServerGenerateBrokerIdTest > testUserConfigAndGeneratedBrokerId 
STARTED

kafka.server.ServerGenerateBrokerIdTest > testUserConfigAndGeneratedBrokerId 
PASSED

kafka.server.ServerGenerateBrokerIdTest > 
testConsistentBrokerIdFromUserConfigAndMetaProps STARTED

kafka.server.ServerGenerateBrokerIdTest > 
testConsistentBrokerIdFromUserConfigAndMetaProps PASSED

kafka.server.DelayedOperationTest > testRequestPurge STARTED

kafka.server.DelayedOperationTest > testRequestPurge PASSED

kafka.server.DelayedOperationTest > testRequestExpiry STARTED

kafka.server.DelayedOperationTest > testRequestExpiry PASSED

kafka.server.DelayedOperationTest > 
shouldReturnNilOperationsOnCancelForKeyWhenKeyDoesntExist STARTED

kafka.server.DelayedOperationTest > 
shouldReturnNilOperationsOnCancelForKeyWhenKeyDoesntExist PASSED

kafka.server.DelayedOperationTest > 
shouldCancelForKeyReturningCancelledOperations STARTED

kafka.server.DelayedOperationTest > 
shouldCancelForKeyReturningCancelledOperations PASSED

kafka.server.DelayedOperationTest > testRequestSatisfaction STARTED

kafka.server.DelayedOperationTest > testRequestSatisfaction PASSED

kafka.server.MultipleListenersWithDefaultJaasContextTest > testProduceConsume 
STARTED

kafka.server.MultipleListenersWithDefaultJaasContextTest > testProduceConsume 
PASSED

kafka.server.ThrottledResponseExpirationTest > testThrottledRequest STARTED

kafka.server.ThrottledResponseExpirationTest > testThrottledRequest PASSED

kafka.server.ThrottledResponseExpirationTest > testExpire STARTED

kafka.server.ThrottledResponseExpirationTest > testExpire PASSED

kafka.server.ReplicaManagerQuotasTest > shouldGetBothMessagesIfQuotasAllow 
STARTED

kafka.server.ReplicaManagerQuotasTest > shouldGetBothMessagesIfQuotasAllow 
PASSED

kafka.server.ReplicaManagerQuotasTest > 
shouldExcludeSubsequentThrottledPartitions STARTED

kafka.server.ReplicaManagerQuotasTest > 
shouldExcludeSubsequentThrottledPartitions PASSED

kafka.server.ReplicaManagerQuotasTest > 
shouldGetNoMessagesIfQuotasExceededOnSubsequentPartitions STARTED

kafka.server.ReplicaManagerQuotasTest > 
shouldGetNoMessagesIfQuotasExceededOnSubsequentPartitions PASSED

kafka.server.ReplicaManagerQuotasTest > shouldIncludeInSyncThrottledReplicas 
STARTED

kafka.server.ReplicaManagerQuotasTest > shouldIncludeInSyncThrottledReplicas 
PASSED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestWithUnsupportedVersion STARTED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestWithUnsupportedVersion PASSED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestBeforeSaslHandshakeRequest STARTED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestBeforeSaslHandshakeRequest PASSED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestAfterSaslHandshakeRequest STARTED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestAfterSaslHandshakeRequest PASSED

kafka.server.ControlledShutdownLeaderSelectorTest > testSelectLeader STARTED

kafka.server.ControlledShutdownLeaderSelectorTest > testSelectLeader PASSED

kafka.server.MetadataCacheTest > 
getTopicMetadataWithNonSupportedSecurityProtocol STARTED

kafka.server.MetadataCacheTest > 
getTopicMetadataWithNonSupportedSecurityProtocol PASSED

kafka.server.MetadataCacheTest > getTopicMetadataIsrNotAvailable STARTED

kafka.server.MetadataCacheTest > getTopicMetadataIsrNotAvailable PASSED

kafka.server.MetadataCacheTest > getTopicMetadata STARTED

kafka.server.MetadataCacheTest > getTopicMetadata PASSED

kafka.server.MetadataCacheTest > getTopicMetadataReplicaNotAvailable STARTED

kafka.server.MetadataCacheTest > getTopicMetadataReplicaNotAvailable PASSED

kafka.server.MetadataCacheTest > getTopicMetadataPartitionLeaderNotAvailable 
STARTED

kafka.server.MetadataCacheTest > getTopicMetadataPartitionLeaderNotAvailable 
PASSED

kafka.server.MetadataCacheTest > getAliveBrokersShouldNotBeMutatedByUpdateCache 
STARTED

kafka.server.MetadataCacheTest > getAliveBrokersShouldNotBeMutatedByUpdateCache 
PASSED

kafka.server.MetadataCacheTest > getTopicMetadataNonExistingTopics STARTED

kafka.server.MetadataCacheTest > getTopicMetadataNonExistingTopics PASSED

kafka.server.FetchRequestTest > testBrokerRespectsPartitionsOrderAndSizeLimits 
STARTED

kafka.server.FetchRequestTest > testBrokerRespectsPartitionsOrderAndSizeLimits 
PASSED

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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[jason] MINOR: Remove unused SecurityProtocol.TRACE

--
[...truncated 2.54 MB...]

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
testNoMessagesSentExceptionFromOverlappingPatterns STARTED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
testNoMessagesSentExceptionFromOverlappingPatterns PASSED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
shouldAddStateStoreToRegexDefinedSource STARTED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
shouldAddStateStoreToRegexDefinedSource PASSED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithZeroSizedCache STARTED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithZeroSizedCache PASSED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithNonZeroSizedCache STARTED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithNonZeroSizedCache PASSED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldCompactTopicsForStateChangelogs STARTED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldCompactTopicsForStateChangelogs PASSED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldUseCompactAndDeleteForWindowStoreChangelogs STARTED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldUseCompactAndDeleteForWindowStoreChangelogs PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceSessionWindows STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceSessionWindows PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduce STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduce PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregate STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregate PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCount STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCount PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldGroupByKey STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldGroupByKey PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountWithInternalStore STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountWithInternalStore PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceWindowed STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceWindowed PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountSessionWindows STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountSessionWindows PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregateWindowed STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregateWindowed PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin PASSED

org.apache.kafka.streams.integration.ResetIntegrationTest > 
testReprocessingFromScratchAfterResetWithIntermediateUserTopic STARTED

org.apache.kafka.streams.integration.ResetIntegrationTest > 
testReprocessingFromScratchAfterResetWithIntermediateUserTopic FAILED
java.lang.AssertionError: expected:<0> but was:<1>
at org.junit.Assert.fail(Assert.java:88)
at org.junit.Assert.failNotEquals(Assert.java:834)
at org.junit.Assert.assertEquals(Assert.java:645)
at org.junit.Assert.assertEquals(Assert.java:631)
at 
org.apache.kafka.streams.integration.ResetIntegrationTest.cleanGlobal(ResetIntegrationTest.java:363)
at 
org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterResetWithIntermediateUserTopic(ResetIntegrationTest.java:172)

org.apache.kafka.streams.integration.ResetIntegrationTest > 
testReprocessingFromScratchAfterResetWithoutIntermediateUserTopic STARTED

org.apac

Please grant me Wiki Access

2017-09-14 Thread chetna chaudhari
Hi,
   Please grant me wiki page access. I need the access to create a KIP for
change .
my wiki id is 'chetna' .

-- 
Thanks and Regards,
Chetna Chaudhari


Re: Please grant me Wiki Access

2017-09-14 Thread Jun Rao
Hi, Chetna,

Thanks for your interest. Just gave you the wiki permission.

Jun


On Thu, Sep 14, 2017 at 2:48 PM, chetna chaudhari  wrote:

> Hi,
>Please grant me wiki page access. I need the access to create a KIP for
> change .
> my wiki id is 'chetna' .
>
> --
> Thanks and Regards,
> Chetna Chaudhari
>


[jira] [Created] (KAFKA-5898) Client consume more than one time a message

2017-09-14 Thread Cyril WIRTH (JIRA)
Cyril WIRTH created KAFKA-5898:
--

 Summary: Client consume more than one time a message
 Key: KAFKA-5898
 URL: https://issues.apache.org/jira/browse/KAFKA-5898
 Project: Kafka
  Issue Type: Bug
  Components: clients
Affects Versions: 0.11.0.0
Reporter: Cyril WIRTH
Priority: Critical
 Attachments: AbstractKafkaConfig.java, consommation.log, 
MyKafkaClient.java, MyProducer.java, production.log

Hello,
I'm using an autocommit consumer, i get some messages, and my 
SESSION_TIMEOUT_MS_CONFIG is smaller than the total message processing time. In 
that case, the consumer consume more than one time a message (twice in the log 
file), how it's possible ? As you can see in the log file, the "msg1" which is 
in the partition 0 is consume in the same time in the thread : KAFKA-0 and 
KAFKA-1, however the KAFKA-1 is assigned to the partition 1 not 0.

KAFKA-0,assigned : [vertx_logger-0]
KAFKA-2,assigned : [vertx_logger-2]
KAFKA-1,assigned : [vertx_logger-1]

Can you help me ?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


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

2017-09-14 Thread Randall Hauch
On the advice of Ismael, I made a few minor changes to a few of the metrics
to adhere to the new pattern of `-rate` and `-total` metric pairs defined
in KIP-187 [1]:

1. Several of the "-rate" metrics were paired with a "-count" metric; the
"-count" metrics were renamed to "-total".
2. Two "-rate" metrics were missing a count total metric, so a "-total"
metric was added.

Please respond if you have any concerns or objections to this minor change.

Best regards,

Randall



[1]
https://cwiki.apache.org/confluence/display/KAFKA/KIP-187+-+Add+cumulative+count+metric+for+all+Kafka+rate+metrics

On Wed, Sep 13, 2017 at 9:53 PM, Randall Hauch  wrote:

> The KIP has passed with three binding +1 votes (Gwen, Sriram, Jason) and
> no -1 or +0 votes.
>
> Thanks to everyone for the feedback.
>
> On Tue, Sep 12, 2017 at 2:48 PM, Jason Gustafson 
> wrote:
>
>> +1. Thanks for the KIP.
>>
>> On Tue, Sep 12, 2017 at 12:42 PM, Sriram Subramanian 
>> wrote:
>>
>> > +1
>> >
>> > On Tue, Sep 12, 2017 at 12:41 PM, Gwen Shapira 
>> wrote:
>> >
>> > > My +1 remains :)
>> > >
>> > > On Tue, Sep 12, 2017 at 12:31 PM Randall Hauch 
>> wrote:
>> > >
>> > > > The KIP was modified (most changes due to reorganization of
>> metrics).
>> > > Feel
>> > > > free to re-vote if you dislike the changes.
>> > > >
>> > > > On Mon, Sep 11, 2017 at 8:40 PM, Sriram Subramanian <
>> r...@confluent.io>
>> > > > wrote:
>> > > >
>> > > > > +1
>> > > > >
>> > > > > On Mon, Sep 11, 2017 at 2:56 PM, Gwen Shapira 
>> > > wrote:
>> > > > >
>> > > > > > +1
>> > > > > >
>> > > > > > Thanks for this. Can't wait for more complete monitoring for
>> > Connect.
>> > > > > >
>> > > > > > On Mon, Sep 11, 2017 at 7:40 AM Randall Hauch > >
>> > > > wrote:
>> > > > > >
>> > > > > > > I'd like to start the vote on KIP-196 to add metrics to the
>> Kafka
>> > > > > Connect
>> > > > > > > framework so the worker processes can be measured. Details are
>> > > here:
>> > > > > > >
>> > > > > > >
>> > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>> > > > > > 196%3A+Add+metrics+to+Kafka+Connect+framework
>> > > > > > >
>> > > > > > > Thanks, and best regards.
>> > > > > > >
>> > > > > > > Randall
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>
>


[jira] [Created] (KAFKA-5899) Create Connect metrics for connectors

2017-09-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5899:


 Summary: Create Connect metrics for connectors
 Key: KAFKA-5899
 URL: https://issues.apache.org/jira/browse/KAFKA-5899
 Project: Kafka
  Issue Type: Sub-task
  Components: KafkaConnect
Affects Versions: 0.11.0.0
Reporter: Randall Hauch
Assignee: Randall Hauch
Priority: Critical
 Fix For: 1.0.0


See KAFKA-2376 for parent task and 
[KIP-196|https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework]
 for the details on the metrics. This subtask is to create the "Connect 
Metrics", and a basic framework for easily adding other Connect metrics.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (KAFKA-5901) Create Connect metrics for source tasks

2017-09-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5901:


 Summary: Create Connect metrics for source tasks
 Key: KAFKA-5901
 URL: https://issues.apache.org/jira/browse/KAFKA-5901
 Project: Kafka
  Issue Type: Sub-task
  Components: KafkaConnect
Affects Versions: 0.11.0.0
Reporter: Randall Hauch
Assignee: Randall Hauch
Priority: Critical
 Fix For: 1.0.0


See KAFKA-2376 for parent task and 
[KIP-196|https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework]
 for the details on the metrics. This subtask is to create the "Source Task 
Metrics".



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (KAFKA-5903) Create Connect metrics for workers

2017-09-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5903:


 Summary: Create Connect metrics for workers
 Key: KAFKA-5903
 URL: https://issues.apache.org/jira/browse/KAFKA-5903
 Project: Kafka
  Issue Type: Sub-task
  Components: KafkaConnect
Affects Versions: 0.11.0.0
Reporter: Randall Hauch
Assignee: Randall Hauch
Priority: Critical
 Fix For: 1.0.0


See KAFKA-2376 for parent task and 
[KIP-196|https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework]
 for the details on the metrics. This subtask is to create the "Worker Metrics".



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (KAFKA-5904) Create Connect metrics for worker rebalances

2017-09-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5904:


 Summary: Create Connect metrics for worker rebalances
 Key: KAFKA-5904
 URL: https://issues.apache.org/jira/browse/KAFKA-5904
 Project: Kafka
  Issue Type: Sub-task
  Components: KafkaConnect
Affects Versions: 0.11.0.0
Reporter: Randall Hauch
Assignee: Randall Hauch
Priority: Critical
 Fix For: 1.0.0


See KAFKA-2376 for parent task and 
[KIP-196|https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework]
 for the details on the metrics. This subtask is to create the "Worker 
Rebalance Metrics".



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (KAFKA-5902) Create Connect metrics for sink tasks

2017-09-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5902:


 Summary: Create Connect metrics for sink tasks
 Key: KAFKA-5902
 URL: https://issues.apache.org/jira/browse/KAFKA-5902
 Project: Kafka
  Issue Type: Sub-task
  Components: KafkaConnect
Affects Versions: 0.11.0.0
Reporter: Randall Hauch
Assignee: Randall Hauch
Priority: Critical
 Fix For: 1.0.0


See KAFKA-2376 for parent task and 
[KIP-196|https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework]
 for the details on the metrics. This subtask is to create the "Sink Task 
Metrics".



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] kafka pull request #3864: KAFKA-5899 Added Connect metrics for connectors

2017-09-14 Thread rhauch
GitHub user rhauch opened a pull request:

https://github.com/apache/kafka/pull/3864

KAFKA-5899 Added Connect metrics for connectors

Added metrics for each connector using Kafka’s existing `Metrics` 
framework. Since this is the first of several changes to add several groups of 
metrics, this change starts by adding a very simple `ConnectMetrics` object 
that is owned by each worker and that makes it easy to define multiple groups 
of metrics, called `ConnectMetricGroup` objects. Each metric group maps to a 
JMX MBean, and each metric within the group maps to an MBean attribute.

Future PRs will build upon this simple pattern to add metrics for source 
and sink tasks, workers, and worker rebalances.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/rhauch/kafka kafka-5899

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3864.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3864


commit 02da9b0ebaa9df70c0dfb34799d3a08b051b9570
Author: Randall Hauch 
Date:   2017-09-13T14:48:14Z

KAFKA-5899 Added Connect metrics for connectors

Added metrics for each connector using Kafka’s existing `Metrics` 
framework. Since this is the first of several changes to add several groups of 
metrics, this change starts by adding a very simple `ConnectMetrics` object 
that is owned by each worker and that makes it easy to define multiple groups 
of metrics, called `ConnectMetricGroup` objects. Each metric group maps to a 
JMX MBean, and each metric within the group maps to an MBean attribute.

Future PRs will build upon this simple pattern to add metrics for source 
and sink tasks, workers, and worker rebalances.




---


[jira] [Created] (KAFKA-5900) Create Connect metrics common to source and sink tasks

2017-09-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-5900:


 Summary: Create Connect metrics common to source and sink tasks
 Key: KAFKA-5900
 URL: https://issues.apache.org/jira/browse/KAFKA-5900
 Project: Kafka
  Issue Type: Sub-task
  Components: KafkaConnect
Affects Versions: 0.11.0.0
Reporter: Randall Hauch
Assignee: Randall Hauch
Priority: Critical
 Fix For: 1.0.0


See KAFKA-2376 for parent task and 
[KIP-196|https://cwiki.apache.org/confluence/display/KAFKA/KIP-196%3A+Add+metrics+to+Kafka+Connect+framework]
 for the details on the metrics. This subtask is to create the "Common Task 
Metrics".



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] kafka pull request #3686: KAFKA-5738: Add cumulative count for rate metrics ...

2017-09-14 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3686


---


[GitHub] kafka pull request #3743: KAFKA-5494: enable idempotence with max.in.flight....

2017-09-14 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3743


---


Build failed in Jenkins: kafka-trunk-jdk9 #11

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[ismael] KAFKA-5738; Add cumulative count for rate metrics (KIP-187)

[jason] KAFKA-5494; Enable idempotence with

--
[...truncated 1.01 MB...]
org.apache.kafka.common.security.kerberos.KerberosNameTest > testParse STARTED

org.apache.kafka.common.security.kerberos.KerberosNameTest > testParse PASSED

org.apache.kafka.common.security.auth.KafkaPrincipalTest > 
testEqualsAndHashCode STARTED

org.apache.kafka.common.security.auth.KafkaPrincipalTest > 
testEqualsAndHashCode PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseOldPrincipalBuilderForPlaintextIfProvided STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseOldPrincipalBuilderForPlaintextIfProvided PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderScram STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderScram PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderGssapi STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderGssapi PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseSessionPeerPrincipalForSsl STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseSessionPeerPrincipalForSsl PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseOldPrincipalBuilderForSslIfProvided STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseOldPrincipalBuilderForSslIfProvided PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testReturnAnonymousPrincipalForPlaintext STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testReturnAnonymousPrincipalForPlaintext PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameOverride STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameOverride PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingOptionValue 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingOptionValue PASSED

org.apache.kafka.common.security.JaasContextTest > testSingleOption STARTED

org.apache.kafka.common.security.JaasContextTest > testSingleOption PASSED

org.apache.kafka.common.security.JaasContextTest > 
testNumericOptionWithoutQuotes STARTED

org.apache.kafka.common.security.JaasContextTest > 
testNumericOptionWithoutQuotes PASSED

org.apache.kafka.common.security.JaasContextTest > testConfigNoOptions STARTED

org.apache.kafka.common.security.JaasContextTest > testConfigNoOptions PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithWrongListenerName STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithWrongListenerName PASSED

org.apache.kafka.common.security.JaasContextTest > testNumericOptionWithQuotes 
STARTED

org.apache.kafka.common.security.JaasContextTest > testNumericOptionWithQuotes 
PASSED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionValue STARTED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionValue PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingLoginModule 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingLoginModule PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingSemicolon STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingSemicolon PASSED

org.apache.kafka.common.security.JaasContextTest > testMultipleOptions STARTED

org.apache.kafka.common.security.JaasContextTest > testMultipleOptions PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForClientWithListenerName STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForClientWithListenerName PASSED

org.apache.kafka.common.security.JaasContextTest > testMultipleLoginModules 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMultipleLoginModules 
PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingControlFlag 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingControlFlag PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameAndFallback STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameAndFallback PASSED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionName STARTED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionName PASSED

org.apache.kafka.common.security.JaasContextTest > testControlFlag STARTED

org.apache.kafka.common.security.JaasContextTest > testControlFlag PASSED

org.apa

[jira] [Created] (KAFKA-5905) Remove PrincipalBuilder and DefaultPrincipalBuilder

2017-09-14 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-5905:
--

 Summary: Remove PrincipalBuilder and DefaultPrincipalBuilder
 Key: KAFKA-5905
 URL: https://issues.apache.org/jira/browse/KAFKA-5905
 Project: Kafka
  Issue Type: Improvement
Reporter: Jason Gustafson
 Fix For: 2.0.0


These classes were deprecated after KIP-189: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-189%3A+Improve+principal+builder+interface+and+add+support+for+SASL,
 which is part of 1.0.0.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[ismael] KAFKA-5738; Add cumulative count for rate metrics (KIP-187)

--
[...truncated 2.04 MB...]
org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderScram PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderGssapi STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderGssapi PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseSessionPeerPrincipalForSsl STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseSessionPeerPrincipalForSsl PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseOldPrincipalBuilderForSslIfProvided STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseOldPrincipalBuilderForSslIfProvided PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testReturnAnonymousPrincipalForPlaintext STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testReturnAnonymousPrincipalForPlaintext PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameOverride STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameOverride PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingOptionValue 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingOptionValue PASSED

org.apache.kafka.common.security.JaasContextTest > testSingleOption STARTED

org.apache.kafka.common.security.JaasContextTest > testSingleOption PASSED

org.apache.kafka.common.security.JaasContextTest > 
testNumericOptionWithoutQuotes STARTED

org.apache.kafka.common.security.JaasContextTest > 
testNumericOptionWithoutQuotes PASSED

org.apache.kafka.common.security.JaasContextTest > testConfigNoOptions STARTED

org.apache.kafka.common.security.JaasContextTest > testConfigNoOptions PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithWrongListenerName STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithWrongListenerName PASSED

org.apache.kafka.common.security.JaasContextTest > testNumericOptionWithQuotes 
STARTED

org.apache.kafka.common.security.JaasContextTest > testNumericOptionWithQuotes 
PASSED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionValue STARTED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionValue PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingLoginModule 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingLoginModule PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingSemicolon STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingSemicolon PASSED

org.apache.kafka.common.security.JaasContextTest > testMultipleOptions STARTED

org.apache.kafka.common.security.JaasContextTest > testMultipleOptions PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForClientWithListenerName STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForClientWithListenerName PASSED

org.apache.kafka.common.security.JaasContextTest > testMultipleLoginModules 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMultipleLoginModules 
PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingControlFlag 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingControlFlag PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameAndFallback STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameAndFallback PASSED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionName STARTED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionName PASSED

org.apache.kafka.common.security.JaasContextTest > testControlFlag STARTED

org.apache.kafka.common.security.JaasContextTest > testControlFlag PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
noAuthorizationIdSpecified STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
noAuthorizationIdSpecified PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdEqualsAuthenticationId STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdEqualsAuthenticationId PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdNotEqualsAuthenticationId STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdNotEqualsAuthenticationId PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingUsernameSaslPlain STARTED

org.apache.kafka.comm

[jira] [Reopened] (KAFKA-5792) Transient failure in KafkaAdminClientTest.testHandleTimeout

2017-09-14 Thread Jason Gustafson (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-5792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jason Gustafson reopened KAFKA-5792:


Still seeing this: 
https://builds.apache.org/blue/organizations/jenkins/kafka-trunk-jdk8/detail/kafka-trunk-jdk8/2021/tests.

> Transient failure in KafkaAdminClientTest.testHandleTimeout
> ---
>
> Key: KAFKA-5792
> URL: https://issues.apache.org/jira/browse/KAFKA-5792
> Project: Kafka
>  Issue Type: Bug
>Reporter: Apurva Mehta
>Assignee: Colin P. McCabe
>  Labels: transient-unit-test-failure
> Fix For: 1.0.0
>
>
> The {{KafkaAdminClientTest.testHandleTimeout}} test occasionally fails with 
> the following:
> {noformat}
> java.util.concurrent.ExecutionException: 
> org.apache.kafka.common.errors.TimeoutException: Timed out waiting for a node 
> assignment.
>   at 
> org.apache.kafka.common.internals.KafkaFutureImpl.wrapAndThrow(KafkaFutureImpl.java:45)
>   at 
> org.apache.kafka.common.internals.KafkaFutureImpl.access$000(KafkaFutureImpl.java:32)
>   at 
> org.apache.kafka.common.internals.KafkaFutureImpl$SingleWaiter.await(KafkaFutureImpl.java:89)
>   at 
> org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:213)
>   at 
> org.apache.kafka.clients.admin.KafkaAdminClientTest.testHandleTimeout(KafkaAdminClientTest.java:356)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:606)
>   at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
>   at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>   at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
>   at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>   at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
>   at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
>   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>   at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.kafka.common.errors.TimeoutException: Timed out waiting 
> for a node assignment.
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[ismael] KAFKA-5738; Add cumulative count for rate metrics (KIP-187)

--
[...truncated 2.52 MB...]
org.apache.kafka.streams.integration.QueryableStateIntegrationTest > 
shouldBeAbleToQueryFilterState PASSED

org.apache.kafka.streams.integration.QueryableStateIntegrationTest > 
shouldBeAbleToQueryStateWithNonZeroSizedCache STARTED

org.apache.kafka.streams.integration.QueryableStateIntegrationTest > 
shouldBeAbleToQueryStateWithNonZeroSizedCache PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftOuterJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftOuterJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerLeftJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerLeftJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterOuterJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterOuterJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerOuterJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerOuterJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftInnerJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftInnerJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerInnerJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerInnerJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerOuterJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerOuterJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterLeftJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterLeftJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterInnerJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterInnerJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerInnerJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerInnerJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftOuterJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftOuterJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterLeftJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterLeftJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftLeftJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftLeftJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterInnerJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterInnerJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftInnerJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftInnerJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerLeftJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerLeftJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftLeftJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftLeftJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterOuterJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterOuterJoinQueryable PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin PASSED

org.apache.kafka.streams.integration.FanoutIntegrationTest > 
shouldFanoutTheInput STARTED

org.apache.kafka.streams.integration.FanoutIntegrationTest > 
shouldFanoutTheInput P

[GitHub] kafka-site pull request #78: MINOR: Add header items

2017-09-14 Thread joel-hamill
GitHub user joel-hamill opened a pull request:

https://github.com/apache/kafka-site/pull/78

MINOR: Add header items



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/joel-hamill/kafka-site joel-hamill/header-nav

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka-site/pull/78.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #78


commit 6360f5f440d2eda97074acc87c6ba227495ca35d
Author: Joel Hamill 
Date:   2017-09-15T00:32:25Z

MINOR: Add header items




---


[GitHub] kafka-site issue #78: MINOR: Add header items

2017-09-14 Thread joel-hamill
Github user joel-hamill commented on the issue:

https://github.com/apache/kafka-site/pull/78
  
ping for review @guozhangwang @dguy


---


[GitHub] kafka pull request #3778: KAFKA-5822: Consistent log formatting of topic par...

2017-09-14 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/3778


---


[jira] [Resolved] (KAFKA-5822) Consistent logging of topic partitions

2017-09-14 Thread Ismael Juma (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ismael Juma resolved KAFKA-5822.

   Resolution: Fixed
Fix Version/s: 1.0.0

> Consistent logging of topic partitions
> --
>
> Key: KAFKA-5822
> URL: https://issues.apache.org/jira/browse/KAFKA-5822
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Jason Gustafson
>Assignee: Jason Gustafson
>  Labels: newbie
> Fix For: 1.0.0
>
>
> In some cases partitions are logged as "[topic,partition]" and in others as 
> "topic-partition." It would be nice to standardize to make searching easier.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[jason] KAFKA-5494; Enable idempotence with

--
[...truncated 2.05 MB...]
org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderScram PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderGssapi STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testPrincipalBuilderGssapi PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseSessionPeerPrincipalForSsl STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseSessionPeerPrincipalForSsl PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseOldPrincipalBuilderForSslIfProvided STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testUseOldPrincipalBuilderForSslIfProvided PASSED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testReturnAnonymousPrincipalForPlaintext STARTED

org.apache.kafka.common.security.auth.DefaultKafkaPrincipalBuilderTest > 
testReturnAnonymousPrincipalForPlaintext PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameOverride STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameOverride PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingOptionValue 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingOptionValue PASSED

org.apache.kafka.common.security.JaasContextTest > testSingleOption STARTED

org.apache.kafka.common.security.JaasContextTest > testSingleOption PASSED

org.apache.kafka.common.security.JaasContextTest > 
testNumericOptionWithoutQuotes STARTED

org.apache.kafka.common.security.JaasContextTest > 
testNumericOptionWithoutQuotes PASSED

org.apache.kafka.common.security.JaasContextTest > testConfigNoOptions STARTED

org.apache.kafka.common.security.JaasContextTest > testConfigNoOptions PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithWrongListenerName STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithWrongListenerName PASSED

org.apache.kafka.common.security.JaasContextTest > testNumericOptionWithQuotes 
STARTED

org.apache.kafka.common.security.JaasContextTest > testNumericOptionWithQuotes 
PASSED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionValue STARTED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionValue PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingLoginModule 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingLoginModule PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingSemicolon STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingSemicolon PASSED

org.apache.kafka.common.security.JaasContextTest > testMultipleOptions STARTED

org.apache.kafka.common.security.JaasContextTest > testMultipleOptions PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForClientWithListenerName STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForClientWithListenerName PASSED

org.apache.kafka.common.security.JaasContextTest > testMultipleLoginModules 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMultipleLoginModules 
PASSED

org.apache.kafka.common.security.JaasContextTest > testMissingControlFlag 
STARTED

org.apache.kafka.common.security.JaasContextTest > testMissingControlFlag PASSED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameAndFallback STARTED

org.apache.kafka.common.security.JaasContextTest > 
testLoadForServerWithListenerNameAndFallback PASSED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionName STARTED

org.apache.kafka.common.security.JaasContextTest > testQuotedOptionName PASSED

org.apache.kafka.common.security.JaasContextTest > testControlFlag STARTED

org.apache.kafka.common.security.JaasContextTest > testControlFlag PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
noAuthorizationIdSpecified STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
noAuthorizationIdSpecified PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdEqualsAuthenticationId STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdEqualsAuthenticationId PASSED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdNotEqualsAuthenticationId STARTED

org.apache.kafka.common.security.plain.PlainSaslServerTest > 
authorizatonIdNotEqualsAuthenticationId PASSED

org.apache.kafka.common.security.authenticator.SaslAuthenticatorTest > 
testMissingUsernameSaslPlain STARTED

org.apache.kafka.common.security.authenticator

[GitHub] kafka pull request #3865: KAFKA-5793: Tighten up the semantics of the OutOfO...

2017-09-14 Thread apurvam
GitHub user apurvam opened a pull request:

https://github.com/apache/kafka/pull/3865

KAFKA-5793: Tighten up the semantics of the OutOfOrderSequenceException

*WIP : Don't review yet, still to add tests*

Description of the solution can be found here: 
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Exactly+Once+-+Solving+the+problem+of+spurious+OutOfOrderSequence+errors



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/apurvam/kafka 
KAFKA-5793-tighten-up-out-of-order-sequence-v2

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3865.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3865


commit fb44876987bd2f75c900b187fdc755da3f85114f
Author: Apurva Mehta 
Date:   2017-09-15T01:01:58Z

Initial commit of the client and server code, with minimal tests




---


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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[jason] KAFKA-5494; Enable idempotence with

--
[...truncated 2.53 MB...]
org.apache.kafka.streams.integration.QueryableStateIntegrationTest > 
shouldBeAbleToQueryFilterState PASSED

org.apache.kafka.streams.integration.QueryableStateIntegrationTest > 
shouldBeAbleToQueryStateWithNonZeroSizedCache STARTED

org.apache.kafka.streams.integration.QueryableStateIntegrationTest > 
shouldBeAbleToQueryStateWithNonZeroSizedCache PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftOuterJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftOuterJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerLeftJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerLeftJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterOuterJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterOuterJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerOuterJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerOuterJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftInnerJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftInnerJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerInnerJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerInnerJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerOuterJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerOuterJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterLeftJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterLeftJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterInnerJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterInnerJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerInnerJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerInnerJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftOuterJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftOuterJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterLeftJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterLeftJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftLeftJoin STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftLeftJoin PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterInnerJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterInnerJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftInnerJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftInnerJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerLeftJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldInnerLeftJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftLeftJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldLeftLeftJoinQueryable PASSED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterOuterJoinQueryable STARTED

org.apache.kafka.streams.integration.KTableKTableJoinIntegrationTest > 
shouldOuterOuterJoinQueryable PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin PASSED

org.apache.kafka.streams.integration.FanoutIntegrationTest > 
shouldFanoutTheInput STARTED

org.apache.kafka.streams.integration.FanoutIntegrationTest > 
shouldFanoutTheInput PASSED

org.apache.kafka.s

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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[ismael] KAFKA-5822; Consistent log formatting of topic partitions

--
[...truncated 2.54 MB...]

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
testNoMessagesSentExceptionFromOverlappingPatterns STARTED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
testNoMessagesSentExceptionFromOverlappingPatterns PASSED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
shouldAddStateStoreToRegexDefinedSource STARTED

org.apache.kafka.streams.integration.RegexSourceIntegrationTest > 
shouldAddStateStoreToRegexDefinedSource PASSED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithZeroSizedCache STARTED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithZeroSizedCache PASSED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithNonZeroSizedCache STARTED

org.apache.kafka.streams.integration.KStreamRepartitionJoinTest > 
shouldCorrectlyRepartitionOnJoinOperationsWithNonZeroSizedCache PASSED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldCompactTopicsForStateChangelogs STARTED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldCompactTopicsForStateChangelogs PASSED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldUseCompactAndDeleteForWindowStoreChangelogs STARTED

org.apache.kafka.streams.integration.InternalTopicIntegrationTest > 
shouldUseCompactAndDeleteForWindowStoreChangelogs PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceSessionWindows STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceSessionWindows PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduce STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduce PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregate STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregate PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCount STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCount PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldGroupByKey STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldGroupByKey PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountWithInternalStore STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountWithInternalStore PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceWindowed STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldReduceWindowed PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountSessionWindows STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldCountSessionWindows PASSED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregateWindowed STARTED

org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest > 
shouldAggregateWindowed PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableLeftJoin PASSED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin STARTED

org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > 
shouldKStreamGlobalKTableJoin PASSED

org.apache.kafka.streams.integration.ResetIntegrationTest > 
testReprocessingFromScratchAfterResetWithIntermediateUserTopic STARTED

org.apache.kafka.streams.integration.ResetIntegrationTest > 
testReprocessingFromScratchAfterResetWithIntermediateUserTopic FAILED
java.lang.AssertionError: expected:<0> but was:<1>
at org.junit.Assert.fail(Assert.java:88)
at org.junit.Assert.failNotEquals(Assert.java:834)
at org.junit.Assert.assertEquals(Assert.java:645)
at org.junit.Assert.assertEquals(Assert.java:631)
at 
org.apache.kafka.streams.integration.ResetIntegrationTest.cleanGlobal(ResetIntegrationTest.java:363)
at 
org.apache.kafka.streams.integration.ResetIntegrationTest.testReprocessingFromScratchAfterResetWithIntermediateUserTopic(ResetIntegrationTest.java:172)

org.apache.kafka.streams.integration.ResetIntegrationTest > 
testReprocessingFromScratchAfterResetWithoutIntermediateUserTopic ST

[GitHub] kafka pull request #3866: KAFKA-5874: TopicCommand should check at least one...

2017-09-14 Thread huxihx
GitHub user huxihx opened a pull request:

https://github.com/apache/kafka/pull/3866

KAFKA-5874: TopicCommand should check at least one parameter is given...

When altering topics, TopicCommand should ensure that at least one of 
parameters in `partitions`, `config` or `delete-config` must be specified.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/huxihx/kafka KAFKA-5874

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3866.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3866


commit 33bfbc741aeff38bfc5956df5a8cf8c9bcf51038
Author: huxihx 
Date:   2017-09-15T02:13:33Z

KAFKA-5874: TopicCommand should check at least one parameter is specified 
when altering topics




---


Kip Write Access

2017-09-14 Thread Richard Yu
Hello, I wish to write a kip. Could you grant me access?

Thanks

(Wiki username is yohan.richard.yu)


system test builder

2017-09-14 Thread Ted Yu
Hi,
When I put the following in the address bar of Chrome:

https://jenkins.confluent.io/job/system-test-kafka-branch-builder

I was told:

This site can’t be reached

Are the tests accessible by the public ?

Thanks


Jenkins build is back to normal : kafka-trunk-jdk9 #12

2017-09-14 Thread Apache Jenkins Server
See 




[GitHub] kafka pull request #3842: KAFKA-5301 Improve exception handling on consumer ...

2017-09-14 Thread ConcurrencyPractitioner
Github user ConcurrencyPractitioner closed the pull request at:

https://github.com/apache/kafka/pull/3842


---


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

2017-09-14 Thread Apache Jenkins Server
See 


Changes:

[ismael] KAFKA-5822; Consistent log formatting of topic partitions

--
[...truncated 931.92 KB...]

kafka.server.ServerGenerateBrokerIdTest > testMultipleLogDirsMetaProps STARTED

kafka.server.ServerGenerateBrokerIdTest > testMultipleLogDirsMetaProps PASSED

kafka.server.ServerGenerateBrokerIdTest > testDisableGeneratedBrokerId STARTED

kafka.server.ServerGenerateBrokerIdTest > testDisableGeneratedBrokerId PASSED

kafka.server.ServerGenerateBrokerIdTest > testUserConfigAndGeneratedBrokerId 
STARTED

kafka.server.ServerGenerateBrokerIdTest > testUserConfigAndGeneratedBrokerId 
PASSED

kafka.server.ServerGenerateBrokerIdTest > 
testConsistentBrokerIdFromUserConfigAndMetaProps STARTED

kafka.server.ServerGenerateBrokerIdTest > 
testConsistentBrokerIdFromUserConfigAndMetaProps PASSED

kafka.server.DelayedOperationTest > testRequestPurge STARTED

kafka.server.DelayedOperationTest > testRequestPurge PASSED

kafka.server.DelayedOperationTest > testRequestExpiry STARTED

kafka.server.DelayedOperationTest > testRequestExpiry PASSED

kafka.server.DelayedOperationTest > 
shouldReturnNilOperationsOnCancelForKeyWhenKeyDoesntExist STARTED

kafka.server.DelayedOperationTest > 
shouldReturnNilOperationsOnCancelForKeyWhenKeyDoesntExist PASSED

kafka.server.DelayedOperationTest > 
shouldCancelForKeyReturningCancelledOperations STARTED

kafka.server.DelayedOperationTest > 
shouldCancelForKeyReturningCancelledOperations PASSED

kafka.server.DelayedOperationTest > testRequestSatisfaction STARTED

kafka.server.DelayedOperationTest > testRequestSatisfaction PASSED

kafka.server.MultipleListenersWithDefaultJaasContextTest > testProduceConsume 
STARTED

kafka.server.MultipleListenersWithDefaultJaasContextTest > testProduceConsume 
PASSED

kafka.server.ThrottledResponseExpirationTest > testThrottledRequest STARTED

kafka.server.ThrottledResponseExpirationTest > testThrottledRequest PASSED

kafka.server.ThrottledResponseExpirationTest > testExpire STARTED

kafka.server.ThrottledResponseExpirationTest > testExpire PASSED

kafka.server.ReplicaManagerQuotasTest > shouldGetBothMessagesIfQuotasAllow 
STARTED

kafka.server.ReplicaManagerQuotasTest > shouldGetBothMessagesIfQuotasAllow 
PASSED

kafka.server.ReplicaManagerQuotasTest > 
shouldExcludeSubsequentThrottledPartitions STARTED

kafka.server.ReplicaManagerQuotasTest > 
shouldExcludeSubsequentThrottledPartitions PASSED

kafka.server.ReplicaManagerQuotasTest > 
shouldGetNoMessagesIfQuotasExceededOnSubsequentPartitions STARTED

kafka.server.ReplicaManagerQuotasTest > 
shouldGetNoMessagesIfQuotasExceededOnSubsequentPartitions PASSED

kafka.server.ReplicaManagerQuotasTest > shouldIncludeInSyncThrottledReplicas 
STARTED

kafka.server.ReplicaManagerQuotasTest > shouldIncludeInSyncThrottledReplicas 
PASSED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestWithUnsupportedVersion STARTED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestWithUnsupportedVersion PASSED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestBeforeSaslHandshakeRequest STARTED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestBeforeSaslHandshakeRequest PASSED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestAfterSaslHandshakeRequest STARTED

kafka.server.SaslApiVersionsRequestTest > 
testApiVersionsRequestAfterSaslHandshakeRequest PASSED

kafka.server.ControlledShutdownLeaderSelectorTest > testSelectLeader STARTED

kafka.server.ControlledShutdownLeaderSelectorTest > testSelectLeader PASSED

kafka.server.MetadataCacheTest > 
getTopicMetadataWithNonSupportedSecurityProtocol STARTED

kafka.server.MetadataCacheTest > 
getTopicMetadataWithNonSupportedSecurityProtocol PASSED

kafka.server.MetadataCacheTest > getTopicMetadataIsrNotAvailable STARTED

kafka.server.MetadataCacheTest > getTopicMetadataIsrNotAvailable PASSED

kafka.server.MetadataCacheTest > getTopicMetadata STARTED

kafka.server.MetadataCacheTest > getTopicMetadata PASSED

kafka.server.MetadataCacheTest > getTopicMetadataReplicaNotAvailable STARTED

kafka.server.MetadataCacheTest > getTopicMetadataReplicaNotAvailable PASSED

kafka.server.MetadataCacheTest > getTopicMetadataPartitionLeaderNotAvailable 
STARTED

kafka.server.MetadataCacheTest > getTopicMetadataPartitionLeaderNotAvailable 
PASSED

kafka.server.MetadataCacheTest > getAliveBrokersShouldNotBeMutatedByUpdateCache 
STARTED

kafka.server.MetadataCacheTest > getAliveBrokersShouldNotBeMutatedByUpdateCache 
PASSED

kafka.server.MetadataCacheTest > getTopicMetadataNonExistingTopics STARTED

kafka.server.MetadataCacheTest > getTopicMetadataNonExistingTopics PASSED

kafka.server.FetchRequestTest > testBrokerRespectsPartitionsOrderAndSizeLimits 
STARTED

kafka.server.FetchRequestTest > testBrokerRespectsPartitionsOrderAndSizeLimits 
PASSED

kafka.server.FetchRequestTest > 
testDownConversionFromBa

MetricsReporter Does Not Receive Auto-generated Client Id

2017-09-14 Thread Kevin Lu
Hi,

Is there a particular reason why the MetricsReporter class's configure()
call is not guaranteed to receive the client id?

The interceptors' configure() methods are guaranteed to have client id
field (either user specified or the kafka auto generated client id such as
producer-1).

We are implementing something similar to Confluent's monitoring
interceptors, except we want the MetricsReporter to be the class to
initiate our monitoring components. Since the MetricsReporter is not
guaranteed a client id, we cannot 100% create a direct mapping between
MetricsReporter and the interceptors of a client if the user does not
specify the client id field.

Regards,
Kevin

-- 
Kevin Li Lu
University of California, Berkeley | Class of 2017
College of Letters & Sciences | B.A. Computer Science
Cell: (408) 609-6238


Re: MetricsReporter Does Not Receive Auto-generated Client Id

2017-09-14 Thread Manikumar
Hi,

I think there is no specific reason. we can pass the clientId to
MetricsReporter
class's configure() method.
Can you pls raise the JIRA?

On Fri, Sep 15, 2017 at 7:44 AM, Kevin Lu 
wrote:

> Hi,
>
> Is there a particular reason why the MetricsReporter class's configure()
> call is not guaranteed to receive the client id?
>
> The interceptors' configure() methods are guaranteed to have client id
> field (either user specified or the kafka auto generated client id such as
> producer-1).
>
> We are implementing something similar to Confluent's monitoring
> interceptors, except we want the MetricsReporter to be the class to
> initiate our monitoring components. Since the MetricsReporter is not
> guaranteed a client id, we cannot 100% create a direct mapping between
> MetricsReporter and the interceptors of a client if the user does not
> specify the client id field.
>
> Regards,
> Kevin
>
> --
> Kevin Li Lu
> University of California, Berkeley | Class of 2017
> College of Letters & Sciences | B.A. Computer Science
> Cell: (408) 609-6238
>


[jira] [Created] (KAFKA-5906) Change metric.reporters configuration key to metrics.reporters to be consistent

2017-09-14 Thread Kevin Lu (JIRA)
Kevin Lu created KAFKA-5906:
---

 Summary: Change metric.reporters configuration key to 
metrics.reporters to be consistent
 Key: KAFKA-5906
 URL: https://issues.apache.org/jira/browse/KAFKA-5906
 Project: Kafka
  Issue Type: Improvement
  Components: config, metrics
Reporter: Kevin Lu
Priority: Minor


The "metric.reporters" configuration key should be consistent with the actual 
classes. Clients have a MetricsReporter.class while the broker has a 
KafkaMetricsReporter.class. 

We have seen quite a few people configure this field incorrectly by setting it 
as "metrics.reporters".

The configuration key could be renamed to "metrics.reporters" to match the 
classes, or the classes can be renamed to MetricReporter.class and 
KafkaMetricReporter.class.




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[GitHub] kafka pull request #3867: MINOR: Change metric.reporters description to use ...

2017-09-14 Thread KevinLiLu
GitHub user KevinLiLu opened a pull request:

https://github.com/apache/kafka/pull/3867

MINOR: Change metric.reporters description to use actual 
MetricsReporter.class name

The `metric.reporters` description in the documentation says to implement 
the `MetricReporter`, but the actual class is `MetricsReporter`. 

The broker configuration section also uses the client description so it 
says to implement the `MetricReporter` interface, but the actual class is 
`KafkaMetricsReporter`.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/KevinLiLu/kafka trunk

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/3867.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3867


commit 9fd55293ae12463d593b7d7ab769ec22c222002d
Author: Kevin Lu 
Date:   2017-09-15T05:35:36Z

MINOR: Change metric.reporters description to use actual 
MetricsReporter.class name




---