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

2017-07-25 Thread Tom Bentley
Hi Ismael,

I assume that's the --partitions for kafka-topics.sh? I must admit I hadn't
considered that tool, only kafka-reassign-partitions.sh. Thanks for
pointing it out, because obviously the AdminClient API needs to be suitable
for reuse in kafka-topics.sh too. Since AdminClient doesn't currently
support an alterTopics() I guess the API should mirror the API for
newTopics(), so something like:

public AlterTopicsResult alterTopics(Set topics);
// where:
public class AlteredTopic {
// constructor etc
public String name();
public int numPartitions();
public int replicationFactor();
Map> replicasAssignment();
}

Note that although NewTopic contains a `Map configs`, I
think the API for changing a topic's config already exists: alterConfigs().

This API is better than having separate methods to set the number of
partitions/replicas and assigning them to brokers, since sometimes people
will want to set the assignment at the same time as changing the
partitions/replicas.

An API like this could then be used by both tools.



On 24 July 2017 at 16:23, Ismael Juma  wrote:

> Hi Tom,
>
> I think it makes sense to keep progress reporting simple in the initial
> version. As you say, time to completion is tricky to compute and it seems
> like it should be tackled as its own KIP.
>
> Regarding waiting for reassignment completion, it's easy enough for people
> to do that via a script, so I don't think we need to add it to the tool.
>
> One question that occurred to me, the tool allows one to add partitions to
> existing topics:
>
> val partitionsOpt = parser.accepts("partitions", "The number of partitions
> for the topic being created or " +
>   "altered (WARNING: If partitions are increased for a topic that has a
> key, the partition logic or ordering of the messages will be affected")
>
> It seems like it may make sense to have that as an AdminClient API as well.
> If we agree to do that, then we need to decide if it should be implemented
> client-side or by adding a protocol API. The former is simpler, but the
> latter would allow non Java clients to use it without duplicating the logic
> of assigning replicas to the new partitions. What are your thoughts?
>
> Ismael
>
> On Sat, Jul 22, 2017 at 10:14 AM, Tom Bentley 
> wrote:
>
> > Thinking about this some more, I release that the proposed API for
> tracking
> > progress is a bit specific to this reassignment use case. A more
> generally
> > useful API would be to be able to find out, for a partition on a broker:
> >
> > * When the broker became a follower
> > * When the broker was last in the ISR
> > * If the broker is not in the ISR, how many messages behind it is
> >
> > That's not enough to be able to calculate a percentage completion of a
> > reassignment though (we don't know how far behind it was when it became a
> > follower). But maybe we could maintain a maximum of how far behind it has
> > fallen behind the ISR, since it became a follower/dropped out of the ISR.
> >
> > btw, we need the middle bullet to cope with following sequence:
> >
> > 1. Start syncing
> > 2. Finish syncing
> > 3. Fall behind (drop out of ISR for some reason)
> > 4. User queries for if reassignment has finished (it has, but just
> looking
> > at the ISR would give the impression that it has not).
> >
> >
> >
> >
> > On 21 July 2017 at 11:09, Tom Bentley  wrote:
> >
> > > Aside: I've started this new DISCUSS thread for KIP-179 since the
> > original
> > > one had the incorrect KIP number 178. The original thread can be found
> > > here: http://mail-archives.apache.org/mod_mbox/kafka-dev/201707.
> > > mbox/%3cCAMd5YszudP+-8z5KTbFh6JscT2p4xFi1=VZWWX+
> > > 5dccpxry...@mail.gmail.com%3e
> > >
> > > I've just updated KIP-179 to support Ismael's request for the command
> to
> > > be able to support progress reporting of an ongoing partition
> > reassignment.
> > >
> > > I'll call out two things which I'm not sure about since I don't yet
> have
> > > much experience of Kafka being used operationally:
> > >
> > > 1. As currently constructed the --progress option could report an
> overall
> > > progress percentage, per-partition percentages and errors. It cannot
> > > provide any kind of time-to-completion estimate. Part of me is loath to
> > do
> > > this, as I'm sure we all remember file transfer dialogs that provide
> > > amusing/baffling time-to-completion estimates. So it might be hard to
> do
> > > _well_. On the other hand I expect the thing people will be interested
> in
> > > will often be "when will it be finished?"
> > >
> > > 2. There is no option for the tool to wait for reassignment
> completion. I
> > > can imagine users might want to script something to happen after the
> > > reassignment is complete, and without some kind of --wait option they
> > will
> > > have to poll for completion "manually". Having a --wait optin and
> putting
> > > this polling in the tool means we have a lot more control over how
> often
> 

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

2017-07-25 Thread Paolo Patierno
Hi,


I was digging into it because I need something like an Admin Client alter API 
for my work on rewriting the TopicCommand tool using them.

The AlterConfigs API is used for changing topic level configuration (i.e. 
retention.ms, retention.bytes and so on).

A new AlterTopic API could be better in order to change topic "high level" 
structure so number of partitions, replication factors and so on.

My opinion is that we need separate API because from my point of view they are 
different settings.


Thanks,

Paolo.


Paolo Patierno
Senior Software Engineer (IoT) @ Red Hat
Microsoft MVP on Windows Embedded & IoT
Microsoft Azure Advisor

Twitter : @ppatierno
Linkedin : paolopatierno
Blog : DevExperience



From: Tom Bentley 
Sent: Tuesday, July 25, 2017 9:02 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use 
AdminClient

Hi Ismael,

I assume that's the --partitions for kafka-topics.sh? I must admit I hadn't
considered that tool, only kafka-reassign-partitions.sh. Thanks for
pointing it out, because obviously the AdminClient API needs to be suitable
for reuse in kafka-topics.sh too. Since AdminClient doesn't currently
support an alterTopics() I guess the API should mirror the API for
newTopics(), so something like:

public AlterTopicsResult alterTopics(Set topics);
// where:
public class AlteredTopic {
// constructor etc
public String name();
public int numPartitions();
public int replicationFactor();
Map> replicasAssignment();
}

Note that although NewTopic contains a `Map configs`, I
think the API for changing a topic's config already exists: alterConfigs().

This API is better than having separate methods to set the number of
partitions/replicas and assigning them to brokers, since sometimes people
will want to set the assignment at the same time as changing the
partitions/replicas.

An API like this could then be used by both tools.



On 24 July 2017 at 16:23, Ismael Juma  wrote:

> Hi Tom,
>
> I think it makes sense to keep progress reporting simple in the initial
> version. As you say, time to completion is tricky to compute and it seems
> like it should be tackled as its own KIP.
>
> Regarding waiting for reassignment completion, it's easy enough for people
> to do that via a script, so I don't think we need to add it to the tool.
>
> One question that occurred to me, the tool allows one to add partitions to
> existing topics:
>
> val partitionsOpt = parser.accepts("partitions", "The number of partitions
> for the topic being created or " +
>   "altered (WARNING: If partitions are increased for a topic that has a
> key, the partition logic or ordering of the messages will be affected")
>
> It seems like it may make sense to have that as an AdminClient API as well.
> If we agree to do that, then we need to decide if it should be implemented
> client-side or by adding a protocol API. The former is simpler, but the
> latter would allow non Java clients to use it without duplicating the logic
> of assigning replicas to the new partitions. What are your thoughts?
>
> Ismael
>
> On Sat, Jul 22, 2017 at 10:14 AM, Tom Bentley 
> wrote:
>
> > Thinking about this some more, I release that the proposed API for
> tracking
> > progress is a bit specific to this reassignment use case. A more
> generally
> > useful API would be to be able to find out, for a partition on a broker:
> >
> > * When the broker became a follower
> > * When the broker was last in the ISR
> > * If the broker is not in the ISR, how many messages behind it is
> >
> > That's not enough to be able to calculate a percentage completion of a
> > reassignment though (we don't know how far behind it was when it became a
> > follower). But maybe we could maintain a maximum of how far behind it has
> > fallen behind the ISR, since it became a follower/dropped out of the ISR.
> >
> > btw, we need the middle bullet to cope with following sequence:
> >
> > 1. Start syncing
> > 2. Finish syncing
> > 3. Fall behind (drop out of ISR for some reason)
> > 4. User queries for if reassignment has finished (it has, but just
> looking
> > at the ISR would give the impression that it has not).
> >
> >
> >
> >
> > On 21 July 2017 at 11:09, Tom Bentley  wrote:
> >
> > > Aside: I've started this new DISCUSS thread for KIP-179 since the
> > original
> > > one had the incorrect KIP number 178. The original thread can be found
> > > here: http://mail-archives.apache.org/mod_mbox/kafka-dev/201707.
> > > mbox/%3cCAMd5YszudP+-8z5KTbFh6JscT2p4xFi1=VZWWX+
> > > 5dccpxry...@mail.gmail.com%3e
> > >
> > > I've just updated KIP-179 to support Ismael's request for the command
> to
> > > be able to support progress reporting of an ongoing partition
> > reassignment.
> > >
> > > I'll call out two things which I'm not sure about since

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

2017-07-25 Thread Ismael Juma
Hi Tom and Paolo,

It's true that increasing the number of partitions is done via the
kafka-topics tool, which is also being converted to use the AdminClient
(but via a different JIRA). I also agree that it would be good to consider
if alterTopics would be a sensible way to support all the use cases or if
it's better to have separate APIs. I think it makes sense to have a single
KIP though as they are related and it will be easier to evaluate as a whole.

Does that make sense?

Ismael

On Tue, Jul 25, 2017 at 10:16 AM, Paolo Patierno  wrote:

> Hi,
>
>
> I was digging into it because I need something like an Admin Client alter
> API for my work on rewriting the TopicCommand tool using them.
>
> The AlterConfigs API is used for changing topic level configuration (i.e.
> retention.ms, retention.bytes and so on).
>
> A new AlterTopic API could be better in order to change topic "high level"
> structure so number of partitions, replication factors and so on.
>
> My opinion is that we need separate API because from my point of view they
> are different settings.
>
>
> Thanks,
>
> Paolo.
>
>
> Paolo Patierno
> Senior Software Engineer (IoT) @ Red Hat
> Microsoft MVP on Windows Embedded & IoT
> Microsoft Azure Advisor
>
> Twitter : @ppatierno
> Linkedin : paolopatierno
> Blog : DevExperience
>
>
> 
> From: Tom Bentley 
> Sent: Tuesday, July 25, 2017 9:02 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use
> AdminClient
>
> Hi Ismael,
>
> I assume that's the --partitions for kafka-topics.sh? I must admit I hadn't
> considered that tool, only kafka-reassign-partitions.sh. Thanks for
> pointing it out, because obviously the AdminClient API needs to be suitable
> for reuse in kafka-topics.sh too. Since AdminClient doesn't currently
> support an alterTopics() I guess the API should mirror the API for
> newTopics(), so something like:
>
> public AlterTopicsResult alterTopics(Set topics);
> // where:
> public class AlteredTopic {
> // constructor etc
> public String name();
> public int numPartitions();
> public int replicationFactor();
> Map> replicasAssignment();
> }
>
> Note that although NewTopic contains a `Map configs`, I
> think the API for changing a topic's config already exists: alterConfigs().
>
> This API is better than having separate methods to set the number of
> partitions/replicas and assigning them to brokers, since sometimes people
> will want to set the assignment at the same time as changing the
> partitions/replicas.
>
> An API like this could then be used by both tools.
>
>
>
> On 24 July 2017 at 16:23, Ismael Juma  wrote:
>
> > Hi Tom,
> >
> > I think it makes sense to keep progress reporting simple in the initial
> > version. As you say, time to completion is tricky to compute and it seems
> > like it should be tackled as its own KIP.
> >
> > Regarding waiting for reassignment completion, it's easy enough for
> people
> > to do that via a script, so I don't think we need to add it to the tool.
> >
> > One question that occurred to me, the tool allows one to add partitions
> to
> > existing topics:
> >
> > val partitionsOpt = parser.accepts("partitions", "The number of
> partitions
> > for the topic being created or " +
> >   "altered (WARNING: If partitions are increased for a topic that
> has a
> > key, the partition logic or ordering of the messages will be affected")
> >
> > It seems like it may make sense to have that as an AdminClient API as
> well.
> > If we agree to do that, then we need to decide if it should be
> implemented
> > client-side or by adding a protocol API. The former is simpler, but the
> > latter would allow non Java clients to use it without duplicating the
> logic
> > of assigning replicas to the new partitions. What are your thoughts?
> >
> > Ismael
> >
> > On Sat, Jul 22, 2017 at 10:14 AM, Tom Bentley 
> > wrote:
> >
> > > Thinking about this some more, I release that the proposed API for
> > tracking
> > > progress is a bit specific to this reassignment use case. A more
> > generally
> > > useful API would be to be able to find out, for a partition on a
> broker:
> > >
> > > * When the broker became a follower
> > > * When the broker was last in the ISR
> > > * If the broker is not in the ISR, how many messages behind it is
> > >
> > > That's not enough to be able to calculate a percentage completion of a
> > > reassignment though (we don't know how far behind it was when it
> became a
> > > follower). But maybe we could maintain a maximum of how far behind it
> has
> > > fallen behind the ISR, since it became a follower/dropped out of the
> ISR.
> > >
> > > btw, we need the middle bullet to cope with following sequence:
> > >
> > > 1. Start syncing
> > > 2. Finish syncing
> > > 3. Fall behind (drop out of ISR for some r

Re: [DISCUSS] 2017 October release planning and release version

2017-07-25 Thread Ismael Juma
On Tue, Jul 18, 2017 at 4:04 PM, Guozhang Wang  wrote:

> I was actually thinking about using dot as well for the rc as well moving
> forward, but I can be convinced if we have some reason to keep it as dash
> as well.


It seems reasonable to use a dash for the RC part as it's a qualifier that
is never part of an official release. Not a huge deal though.

Ismael


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

2017-07-25 Thread Tom Bentley
Hi Ismael and Paolo,

I've updated my KIP [1] to describe an alterTopics() API which would work
for kafka-reassign-partitions.sh. It's still a bit rough, but should be a
good basis for a KIP to cover both tools.

As a first step, if Paolo could review this and check it's compatible with
what he needs for the kafka-topics.sh tool that would be great. Then we can
add to describe his changes to the rest of the kafka-topics.sh tool,
assuming Paolo is happy with this arrangement, Paolo?

Cheers,

Tom

[1]
https://cwiki.apache.org/confluence/display/KAFKA/KIP-179+-+Change+ReassignPartitionsCommand+to+use+AdminClient

On 25 July 2017 at 11:38, Ismael Juma  wrote:

> Hi Tom and Paolo,
>
> It's true that increasing the number of partitions is done via the
> kafka-topics tool, which is also being converted to use the AdminClient
> (but via a different JIRA). I also agree that it would be good to consider
> if alterTopics would be a sensible way to support all the use cases or if
> it's better to have separate APIs. I think it makes sense to have a single
> KIP though as they are related and it will be easier to evaluate as a
> whole.
>
> Does that make sense?
>
> Ismael
>
> On Tue, Jul 25, 2017 at 10:16 AM, Paolo Patierno 
> wrote:
>
> > Hi,
> >
> >
> > I was digging into it because I need something like an Admin Client alter
> > API for my work on rewriting the TopicCommand tool using them.
> >
> > The AlterConfigs API is used for changing topic level configuration (i.e.
> > retention.ms, retention.bytes and so on).
> >
> > A new AlterTopic API could be better in order to change topic "high
> level"
> > structure so number of partitions, replication factors and so on.
> >
> > My opinion is that we need separate API because from my point of view
> they
> > are different settings.
> >
> >
> > Thanks,
> >
> > Paolo.
> >
> >
> > Paolo Patierno
> > Senior Software Engineer (IoT) @ Red Hat
> > Microsoft MVP on Windows Embedded & IoT
> > Microsoft Azure Advisor
> >
> > Twitter : @ppatierno
> > Linkedin : paolopatierno
> > Blog : DevExperience
> >
> >
> > 
> > From: Tom Bentley 
> > Sent: Tuesday, July 25, 2017 9:02 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use
> > AdminClient
> >
> > Hi Ismael,
> >
> > I assume that's the --partitions for kafka-topics.sh? I must admit I
> hadn't
> > considered that tool, only kafka-reassign-partitions.sh. Thanks for
> > pointing it out, because obviously the AdminClient API needs to be
> suitable
> > for reuse in kafka-topics.sh too. Since AdminClient doesn't currently
> > support an alterTopics() I guess the API should mirror the API for
> > newTopics(), so something like:
> >
> > public AlterTopicsResult alterTopics(Set topics);
> > // where:
> > public class AlteredTopic {
> > // constructor etc
> > public String name();
> > public int numPartitions();
> > public int replicationFactor();
> > Map> replicasAssignment();
> > }
> >
> > Note that although NewTopic contains a `Map configs`, I
> > think the API for changing a topic's config already exists:
> alterConfigs().
> >
> > This API is better than having separate methods to set the number of
> > partitions/replicas and assigning them to brokers, since sometimes people
> > will want to set the assignment at the same time as changing the
> > partitions/replicas.
> >
> > An API like this could then be used by both tools.
> >
> >
> >
> > On 24 July 2017 at 16:23, Ismael Juma  wrote:
> >
> > > Hi Tom,
> > >
> > > I think it makes sense to keep progress reporting simple in the initial
> > > version. As you say, time to completion is tricky to compute and it
> seems
> > > like it should be tackled as its own KIP.
> > >
> > > Regarding waiting for reassignment completion, it's easy enough for
> > people
> > > to do that via a script, so I don't think we need to add it to the
> tool.
> > >
> > > One question that occurred to me, the tool allows one to add partitions
> > to
> > > existing topics:
> > >
> > > val partitionsOpt = parser.accepts("partitions", "The number of
> > partitions
> > > for the topic being created or " +
> > >   "altered (WARNING: If partitions are increased for a topic that
> > has a
> > > key, the partition logic or ordering of the messages will be affected")
> > >
> > > It seems like it may make sense to have that as an AdminClient API as
> > well.
> > > If we agree to do that, then we need to decide if it should be
> > implemented
> > > client-side or by adding a protocol API. The former is simpler, but the
> > > latter would allow non Java clients to use it without duplicating the
> > logic
> > > of assigning replicas to the new partitions. What are your thoughts?
> > >
> > > Ismael
> > >
> > > On Sat, Jul 22, 2017 at 10:14 AM, Tom Bentley 
> > > wrote:
> > >
>

[jira] [Created] (KAFKA-5635) KIP-181 Kafka-Connect integrate with kafka ReST Proxy

2017-07-25 Thread Dhananjay Patkar (JIRA)
Dhananjay Patkar created KAFKA-5635:
---

 Summary: KIP-181 Kafka-Connect integrate with kafka ReST Proxy
 Key: KAFKA-5635
 URL: https://issues.apache.org/jira/browse/KAFKA-5635
 Project: Kafka
  Issue Type: Improvement
  Components: KafkaConnect
Reporter: Dhananjay Patkar


Kafka connect currently uses kafka clients which directly connect to kafka 
brokers. 
In a use case, wherein I have many kafka connect [producers] running remotely 
its a challenge to configure broker information on every connect agent.

Also, in case of IP change [upgrade or cluster re-creation], we need to update 
every remote connect configuration.

If kafka connect source connectors talk to ReST endpoint then client is unaware 
of broker details. This way we can transparently upgrade / re-create kafka 
cluster as long as ReST endpoint remains same.




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


[GitHub] kafka pull request #3572: MINOR: Remove unused GroupState.state field

2017-07-25 Thread ijuma
GitHub user ijuma opened a pull request:

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

MINOR: Remove unused GroupState.state field

This field doesn't seem to be used and the value for
`AwaitingSync` seems to be wrong (it seems like it
should have been `2` instead of `5`).

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

$ git pull https://github.com/ijuma/kafka remove-unused-group-state-field

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

https://github.com/apache/kafka/pull/3572.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 #3572


commit 7d42953c769a3e03c4811777fa3a2ecba4322a08
Author: Ismael Juma 
Date:   2017-07-25T13:19:30Z

MINOR: Remove unused GroupState.state field




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


Re: [DISCUSS] KIP-180: Add a broker metric specifying the number of consumer group rebalances in progress

2017-07-25 Thread Ismael Juma
Hi Guozhang,

Thanks for the clarification. The naming does seem a bit unclear. Maybe
`PreparingRebalance` could be `StartingRebalance` or something that makes
it clear that it is part of the rebalance instead of a step before the
actual rebalance. `AwaitingSync` could also be `CompletingRebalance`, but
not sure if that's better.

Ismael

On Mon, Jul 24, 2017 at 7:02 PM, Guozhang Wang  wrote:

> Actually Rebalancing includes two steps, and we name them PrepareRebalance
> and WaitSync (arguably they may not be the best names). But these two steps
> together should be treated as the complete rebalance cycle.
>
>
> Guozhang
>
> On Mon, Jul 24, 2017 at 10:46 AM, Colin McCabe  wrote:
>
> > Hi all,
> >
> > I think maybe it makes sense to rename the "PreparingRebalance" consumer
> > group state to "Rebalancing."  To me, "Preparing" implies that there
> > will be a later "rebalance" state that follows-- but there is not.
> > Since we're now exposing this state name publicly in these metrics,
> > perhaps it makes sense to do this rename now.  Thoughts?
> >
> > best,
> > Colin
> >
> >
> > On Fri, Jul 21, 2017, at 13:52, Colin McCabe wrote:
> > > That's a good point.  I revised the KIP to add metrics for all the
> group
> > > states.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > On Fri, Jul 21, 2017, at 12:08, Guozhang Wang wrote:
> > > > Ah, that's right Jason.
> > > >
> > > > With that I can be convinced to add one metric per each state.
> > > >
> > > > Guozhang
> > > >
> > > > On Fri, Jul 21, 2017 at 11:44 AM, Jason Gustafson <
> ja...@confluent.io>
> > > > wrote:
> > > >
> > > > > >
> > > > > > "Dead" and "Empty" states are transient: groups usually only
> > leaves in
> > > > > this
> > > > > > state for a short while and then being deleted or transited to
> > other
> > > > > > states.
> > > > >
> > > > >
> > > > > This is not strictly true for the "Empty" state which we also use
> to
> > > > > represent simple groups which only use the coordinator to store
> > offsets. I
> > > > > think we may as well cover all the states if we're going to cover
> > any of
> > > > > them specifically.
> > > > >
> > > > > -Jason
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Jul 21, 2017 at 9:45 AM, Guozhang Wang  >
> > wrote:
> > > > >
> > > > > > My two cents:
> > > > > >
> > > > > > "Dead" and "Empty" states are transient: groups usually only
> > leaves in
> > > > > this
> > > > > > state for a short while and then being deleted or transited to
> > other
> > > > > > states.
> > > > > >
> > > > > > Since we have the existing "*NumGroups*" metric, `*NumGroups -
> > > > > > **NumGroupsRebalancing
> > > > > > - **NumGroupsAwaitingSync`* should cover the above three, where
> > "Stable"
> > > > > > should be contributing most of the counts: If we have a bug that
> > causes
> > > > > the
> > > > > > num.Dead / Empty to keep increasing, then we would observe
> > `NumGroups`
> > > > > keep
> > > > > > increasing which should be sufficient for alerting. And trouble
> > shooting
> > > > > of
> > > > > > the issue could be relying on the log4j.
> > > > > >
> > > > > > *Guozhang*
> > > > > >
> > > > > > On Fri, Jul 21, 2017 at 7:19 AM, Ismael Juma 
> > wrote:
> > > > > >
> > > > > > > Thanks for the KIP, Colin. This will definitely be useful. One
> > > > > question:
> > > > > > > would it be useful to have a metric for for the number of
> groups
> > in
> > > > > each
> > > > > > > possible state? The KIP suggests "PreparingRebalance" and
> > > > > "AwaitingSync".
> > > > > > > That leaves "Stable", "Dead" and "Empty". Are those not useful?
> > > > > > >
> > > > > > > Ismael
> > > > > > >
> > > > > > > On Thu, Jul 20, 2017 at 6:52 PM, Colin McCabe <
> > cmcc...@apache.org>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > I posted "KIP-180: Add a broker metric specifying the number
> of
> > > > > > consumer
> > > > > > > > group rebalances in progress" for discussion:
> > > > > > > >
> > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > > 180%3A+Add+a+broker+metric+specifying+the+number+of+
> > > > > > > > consumer+group+rebalances+in+progress
> > > > > > > >
> > > > > > > > Check it out.
> > > > > > > >
> > > > > > > > cheers,
> > > > > > > > Colin
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> >
>
>
>
> --
> -- Guozhang
>


Re: [DISCUSS] 2017 October release planning and release version

2017-07-25 Thread Guozhang Wang
Makes sense, have updated the wiki page to use dashes for rc numbers.

Guozhang

On Tue, Jul 25, 2017 at 3:40 AM, Ismael Juma  wrote:

> On Tue, Jul 18, 2017 at 4:04 PM, Guozhang Wang  wrote:
>
> > I was actually thinking about using dot as well for the rc as well moving
> > forward, but I can be convinced if we have some reason to keep it as dash
> > as well.
>
>
> It seems reasonable to use a dash for the RC part as it's a qualifier that
> is never part of an official release. Not a huge deal though.
>
> Ismael
>



-- 
-- Guozhang


[jira] [Created] (KAFKA-5636) DSL: allow sliding windows to be used directly (i.e. not just implicitly when doing joins)

2017-07-25 Thread Michael Noll (JIRA)
Michael Noll created KAFKA-5636:
---

 Summary: DSL: allow sliding windows to be used directly (i.e. not 
just implicitly when doing joins)
 Key: KAFKA-5636
 URL: https://issues.apache.org/jira/browse/KAFKA-5636
 Project: Kafka
  Issue Type: Improvement
  Components: streams
Affects Versions: 0.11.0.0
Reporter: Michael Noll
Priority: Minor


We support three windowing types in the DSL right now:

* Tumbling windows
* Hopping windows (note: some stream processing tools call these "sliding 
windows")
* Session windows

Some users have expressed the need for sliding windows.  We already support 
sliding windows because they are used for joins, but we don't expose sliding 
windows directly through the API/DSL.



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


[jira] [Created] (KAFKA-5637) Document compatibility and release policies

2017-07-25 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-5637:
--

 Summary: Document compatibility and release policies
 Key: KAFKA-5637
 URL: https://issues.apache.org/jira/browse/KAFKA-5637
 Project: Kafka
  Issue Type: Improvement
Reporter: Ismael Juma
 Fix For: 1.0.0


We should document our compatibility and release policies in one place so that 
people have the correct expectations. This is generally important, but more so 
now that we are releasing 1.0.0.

More details to come.



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


Re: [DISCUSS] 2017 October release planning and release version

2017-07-25 Thread Ismael Juma
Thanks Sonke, that's great. I filed an initial JIRA:

https://issues.apache.org/jira/browse/KAFKA-5637

As per our offline conversation, you captured the thread discussion
already, so feel free to flesh out the JIRA.

Ismael

On Mon, Jul 24, 2017 at 8:45 PM, Sönke Liebau <
soenke.lie...@opencore.com.invalid> wrote:

> I am happy to volunteer for working on documenting the release logistics
> and related topics if someone is needed for that part.
>
> Best regards,
> Sönke
>
> Am 24.07.2017 9:28 nachm. schrieb "Guozhang Wang" :
>
> > Thanks for everyone who has shared their thoughts and feedbacks to this
> > discussion. Here is my conclusion:
> >
> > 1. So far everyone has agreed on naming the next major release version to
> > 1.0; if I did not hear any other opinions by the end of today PDT time I
> > will name the next release as 1.0.0 and update the release wiki page
> > correspondingly.
> >
> > 2. People have also expressed expectations to some of the release
> logistics
> > while we are releasing 1.0.0, including old version support period,
> upgrade
> > support period, bug fix / security fix backport policies etc. Ismael has
> a
> > detailed summary on these in this email thread and we will look for
> someone
> > to drive the documentation of these in parallel with the 1.0.0 release
> > process.
> >
> >
> > Guozhang
> >
> >
> > On Mon, Jul 24, 2017 at 11:45 AM, Becket Qin 
> wrote:
> >
> > > +1 on 1.0. I think it makes a lot of sense given the point Kafka is
> now.
> > To
> > > me Kafka has absolutely reached 1.0 in terms of features/functions.
> That
> > > said, I do share the same opinion with others that the usablity and
> > > policies might still need some improvement to meet the 1.0 standard.
> > >
> > > On Sat, Jul 22, 2017 at 7:21 AM, Matthias J. Sax <
> matth...@confluent.io>
> > > wrote:
> > >
> > > > I agree with Ismael. If we go for 1.0 (what I do support), we need to
> > > > meet people's expectations and should document all policies and
> > > > guarantees explicitly. We might also consider to support older
> releases
> > > > longer and do bug fix release for older releases, too.
> > > >
> > > > Other projects (like Flink) do a fantastic job with this regard and
> we
> > > > should learn from them.
> > > >
> > > > -Matthias
> > > >
> > > > On 7/21/17 9:50 PM, Guozhang Wang wrote:
> > > > > That's fair enough too.
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Fri, Jul 21, 2017 at 12:13 PM, Ismael Juma 
> > > wrote:
> > > > >
> > > > >> Yes, I agree that the choice of version number can be done
> > separately.
> > > > >> That's why I said I'd file a separate JIRA for the documentation
> > > > >> improvements. Having said that, there are some expectations that
> > > people
> > > > >> have for projects that have reached 1.0.0 and we should try to
> > > allocate
> > > > >> time for the important ones.
> > > > >>
> > > > >> Ismael
> > > > >>
> > > > >> On 21 Jul 2017 8:07 pm, "Guozhang Wang" 
> wrote:
> > > > >>
> > > > >>> Thanks Ismael. I agree with you on all these points, and for some
> > of
> > > > >> these
> > > > >>> points like 3) we never have a written-down policy though in
> > practice
> > > > we
> > > > >>> tend to follow some patterns.
> > > > >>>
> > > > >>> To me deciding what's the version number of the next major
> release
> > > does
> > > > >> not
> > > > >>> necessarily mean we need now to change any of these or to set the
> > > hard
> > > > >>> rules along with it; I'd like to keep them as two separate
> > > discussions
> > > > as
> > > > >>> they seem semi-orthogonal to me.
> > > > >>>
> > > > >>>
> > > > >>> Guozhang
> > > > >>>
> > > > >>>
> > > > >>> On Fri, Jul 21, 2017 at 8:44 AM, Ismael Juma 
> > > > wrote:
> > > > >>>
> > > >  On the topic of documentation, we should also document which
> > > releases
> > > > >> are
> > > >  still supported and which are not. There a few factors to
> > consider:
> > > > 
> > > >  1. Which branches receive bug fixes. We typically backport fixes
> > to
> > > > the
> > > > >>> two
> > > >  most recent stable branches (the most recent stable branch
> > typically
> > > > >> gets
> > > >  more backports than the older one).
> > > > 
> > > >  2. Which branches receive security fixes. This could be the same
> > as
> > > > >> `1`,
> > > >  but we could attempt to backport more aggressively for security
> > > fixes
> > > > >> as
> > > >  they tend to be rare (so far at least) and the impact could be
> > > severe.
> > > > 
> > > >  3. The release policy for stable branches. We tend to stop
> > releasing
> > > > >>> from a
> > > >  given stable branch before we stop backporting fixes. Maybe
> that's
> > > OK,
> > > > >>> but
> > > >  it would be good to document how we decide that a bug fix
> release
> > is
> > > >  needed.
> > > > 
> > > >  4. How long are direct upgrades supported for. During the
> > time-based
> > > >  releases discussion, we agreed to support direct 

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

2017-07-25 Thread Paolo Patierno
Hi Tom,


I haven taken a look to the updated KIP, some thoughts :


  *   in the "Public Interfaces" section you wrote 
alterTopics(Set) but then a collection is used (instead of a set) 
in the Proposed Changes section. I'm ok with collection.
  *   in the summary of the alterTopics method you say "The request can change 
the number of partitions, replication factor and/or the partition assignments." 
I think that the "and/or" is misleading (at least for me). For the TopicCommand 
tool you can specify partitions AND replication factor ... OR partition 
assignments (but not for example partitions AND replication factor AND 
partition assignments). Maybe it could be "The request can change the number of 
partitions and the related replication factor or specifying a partition 
assignments."
  *   I know that it would be a breaking change in the Admin Client API but why 
having an AlteredTopic class which is quite similar to the already existing 
NewTopic class ? I know that using NewTopic for the alter method could be 
misleading. What about renaming NewTopic in something like AdminTopic ? At same 
time it means that the TopicDetails class (which you can get from the current 
NewTopic) should be outside the CreateTopicsRequest because it could be used in 
the AlterTopicsRequest as well.
  *   A typo in the ReplicaStatus : gpartition() instead of partition()
  *   In the AlterTopicRequets
 *   the replication factor should be INT16
 *   I would use same fields name as CreateTopicsRequest (they are quite 
similar)
  *   What's the broker id in the ReplicaStatusRequest ?
  *   Thinking aloud, could make sense having "Partition" in the 
ReplicaStatusRequest as an array so that I can specify in only one request the 
status for partitions I'm interested in, in order to avoid e request for each 
partition for the same topic. Maybe empty array could mean .. "give me status 
for all partitions of this topic". Of course it means that the 
ReplicaStatusResponse should change because we should have an array with 
partition, broker, lag and so on

Thanks,
Paolo


Paolo Patierno
Senior Software Engineer (IoT) @ Red Hat
Microsoft MVP on Windows Embedded & IoT
Microsoft Azure Advisor

Twitter : @ppatierno
Linkedin : paolopatierno
Blog : DevExperience



From: Tom Bentley 
Sent: Tuesday, July 25, 2017 11:07 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use 
AdminClient

Hi Ismael and Paolo,

I've updated my KIP [1] to describe an alterTopics() API which would work
for kafka-reassign-partitions.sh. It's still a bit rough, but should be a
good basis for a KIP to cover both tools.

As a first step, if Paolo could review this and check it's compatible with
what he needs for the kafka-topics.sh tool that would be great. Then we can
add to describe his changes to the rest of the kafka-topics.sh tool,
assuming Paolo is happy with this arrangement, Paolo?

Cheers,

Tom

[1]
https://cwiki.apache.org/confluence/display/KAFKA/KIP-179+-+Change+ReassignPartitionsCommand+to+use+AdminClient

On 25 July 2017 at 11:38, Ismael Juma  wrote:

> Hi Tom and Paolo,
>
> It's true that increasing the number of partitions is done via the
> kafka-topics tool, which is also being converted to use the AdminClient
> (but via a different JIRA). I also agree that it would be good to consider
> if alterTopics would be a sensible way to support all the use cases or if
> it's better to have separate APIs. I think it makes sense to have a single
> KIP though as they are related and it will be easier to evaluate as a
> whole.
>
> Does that make sense?
>
> Ismael
>
> On Tue, Jul 25, 2017 at 10:16 AM, Paolo Patierno 
> wrote:
>
> > Hi,
> >
> >
> > I was digging into it because I need something like an Admin Client alter
> > API for my work on rewriting the TopicCommand tool using them.
> >
> > The AlterConfigs API is used for changing topic level configuration (i.e.
> > retention.ms, retention.bytes and so on).
> >
> > A new AlterTopic API could be better in order to change topic "high
> level"
> > structure so number of partitions, replication factors and so on.
> >
> > My opinion is that we need separate API because from my point of view
> they
> > are different settings.
> >
> >
> > Thanks,
> >
> > Paolo.
> >
> >
> > Paolo Patierno
> > Senior Software Engineer (IoT) @ Red Hat
> > Microsoft MVP on Windows Embedded & IoT
> > Microsoft Azure Advisor
> >
> > Twitter : @ppatierno
> > Linkedin : paolopatierno
> > Blog : DevExperience
> >
> >
> > 
> > From: Tom Bentley 
> > Sent: Tuesday, July 25, 2017 9:02 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use
> > AdminClient
> >
> 

Re: [VOTE] KIP-167 (Addendum): Add interface for the state store restoration process

2017-07-25 Thread Sriram Subramanian
+1

On Fri, Jul 21, 2017 at 12:08 PM, Guozhang Wang  wrote:

> +1
>
> On Thu, Jul 20, 2017 at 11:00 PM, Matthias J. Sax 
> wrote:
>
> > +1
> >
> > On 7/20/17 4:22 AM, Bill Bejeck wrote:
> > > Hi,
> > >
> > > After working on the PR for this KIP I discovered that we need to add
> and
> > > additional parameter (TopicPartition) to the StateRestoreListener
> > interface
> > > methods.
> > >
> > > The addition of the TopicPartition is required as the
> > StateRestoreListener
> > > is for the entire application, thus all tasks with recovering state
> > stores
> > > call the same listener instance.  The TopicPartition is needed to
> > > disambiguate the progress of the state store recovery.
> > >
> > > For those that have voted before, please review the updated KIP
> > >  > 167:+Add+interface+for+the+state+store+restoration+process>
> > > and
> > > re-vote.
> > >
> > > Thanks,
> > > Bill
> > >
> >
> >
>
>
> --
> -- Guozhang
>


Re: [DISCUSS] KIP-175: Additional '--describe' views for ConsumerGroupCommand

2017-07-25 Thread Vahid S Hashemian
Hi,

If there is no further feedback on this KIP, I'll start the vote tomorrow.

Thanks.
--Vahid



From:   Vahid S Hashemian/Silicon Valley/IBM
To: dev , "Kafka User" 
Date:   07/03/2017 04:06 PM
Subject:[DISCUSS] KIP-175: Additional '--describe' views for 
ConsumerGroupCommand


Hi,

I created KIP-175 to make some improvements to the ConsumerGroupCommand 
tool.
The KIP can be found here: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-175%3A+Additional+%27--describe%27+views+for+ConsumerGroupCommand

Your review and feedback is welcome!

Thanks.
--Vahid






[jira] [Created] (KAFKA-5638) Inconsistency in consumer group related ACLs

2017-07-25 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-5638:
--

 Summary: Inconsistency in consumer group related ACLs
 Key: KAFKA-5638
 URL: https://issues.apache.org/jira/browse/KAFKA-5638
 Project: Kafka
  Issue Type: Bug
  Components: security
Affects Versions: 0.11.0.0
Reporter: Vahid Hashemian
Assignee: Vahid Hashemian
Priority: Minor


Users can see all groups in the cluster (using consumer group’s {{--list}} 
option) provided that they have {{Describe}} access to the cluster. It would 
make more sense to modify that experience and limit what is listed in the 
output to only those groups they have {{Describe}} access to. The reason is, 
almost everything else is accessible by a user only if the access is 
specifically granted (through ACL {{--add}}); and this scenario should not be 
an exception. The potential change would be updating the minimum required 
permission of {{ListGroup}} from {{Describe (Cluster)}} to {{Describe (Group)}}.

We can also look at this issue from a different angle: A user with {{Read}} 
access to a group can describe the group, but the same user would not see 
anything when listing groups (assuming there is no {{Describe}} access to the 
cluster). It makes more sense for this user to be able to list all groups s/he 
can already describe.

It would be great to know if any user is relying on the existing behavior 
(listing all consumer groups using a {{Describe (Cluster)}} ACL.



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


[GitHub] kafka pull request #3515: MINOR: Make streams quick start more interactive

2017-07-25 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Created] (KAFKA-5639) Enhance DescribeGroups API to return additional group information

2017-07-25 Thread Vahid Hashemian (JIRA)
Vahid Hashemian created KAFKA-5639:
--

 Summary: Enhance DescribeGroups API to return additional group 
information
 Key: KAFKA-5639
 URL: https://issues.apache.org/jira/browse/KAFKA-5639
 Project: Kafka
  Issue Type: Improvement
  Components: core
Reporter: Vahid Hashemian
Assignee: Vahid Hashemian
Priority: Minor


The 
[{{DescribeGroups}}|https://kafka.apache.org/protocol#The_Messages_DescribeGroups]
 API v1 currently returns this information for each consumer group:
* {{error_code}}
* {{group_id}}
* {{state}}
* {{protocol_type}}
* {{protocol}}
* {{members}}

There are additional info in a {{GroupMetadata}} object on the server side, 
some of which could be useful if exposed via the {{DescribeGroups}} API. Here 
are some examples:
* {{generationId}}
* {{leaderId}}
* {{numOffsets}}
* {{hasOffsets}}

Enhancing the API with this additional info means improving the existing tools 
that make use of the API. For example, using this additional info, the consumer 
group command's {{\-\-describe}} output will provide more information about 
each consumer group to help with its monitoring / troubleshooting / 



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


[GitHub] kafka pull request #3573: KAFKA-5630; The consumer should block on courrupt ...

2017-07-25 Thread becketqin
GitHub user becketqin opened a pull request:

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

KAFKA-5630; The consumer should block on courrupt records and keeping throw 
exception

This patch handles the case that a CorruptRecordException is thrown from 
the iterator directly.

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

$ git pull https://github.com/becketqin/kafka KAFKA-5630

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

https://github.com/apache/kafka/pull/3573.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 #3573


commit e51ddce3ddacefee7101b95021a3af5da32bcff4
Author: Jiangjie Qin 
Date:   2017-07-25T18:20:09Z

KAFKA-5630; The consumer should block on courrupt records and keeping throw 
exception.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


Re: Kafka Connect suggestion before creating new KIP

2017-07-25 Thread Ewen Cheslack-Postava
On Mon, Jul 24, 2017 at 1:50 PM, Florian Hussonnois 
wrote:

> Hi all,
>
> Here is two suggestions for the Kafka Connect API that I prefer to present
> you first before creating KIPs.
>
> 1/
> Currently, in Kafka Connect we distinguish two configuration types: the
> worgker config and the connector config. The connector config is accessible
> from the Connector class through th method "*void start(Map
> props)*" but developers can't access the originals worker configuration
> (i.e bootstrap.servers, etc).
>
> I think some use cases could require an internal producer or consumer
> created from the Connector/Task instance.
>
> For example in order to  :
>  - To consume a topic for special events which will trigger tasks
> reconfiguration
>  - To produce messages in order to track tasks progression (the messages
> could be consumed by a KafkaStreams app in order to expose monitoring UI).
>
> Currently, the only way to do this is to define the kafka configuration in
> each configuration but this is not really developer friendly.
>

I think we'd want to be careful about doing this. If you run connect like a
service, you may need to grant the service relatively broad permissions in
Kafka to support many users. But you still may want to lock down what any
given user of the service can do. We currently don't have ACLs (or even
authenticated REST API access), but I know that there are requests for this
functionality. If we just exposed the entire worker config to connectors,
they'd be able to do anything the worker has permission for.

By the way, for your second use case, there's no reason you can't do that
today. Just emit more producer records but point them at a different topic
and


>
> 2/
> Secondly, it could be useful to have a method returning the connector name
> accessible from the Connector class. For example, the connector name could
> be used as an identifier for retrieving sensitives settings from a
> configuration management server (for example Consul).
>
> Do you think this two minor suggestions can deserved KIPs ?
>

This is small and seems reasonable. You could also, of course, just save
the configuration or that specific property when connector.start(config) is
invoked. A more general extension would be to have ConnectorContext expose
the entire original config instead of having a method specifically for the
name.

-Ewen


>
> Thanks very much.
>
> Florian.
>
>
> --
> Florian HUSSONNOIS
>


[GitHub] kafka pull request #3574: HOTFIX: handle commit failed exception on stream t...

2017-07-25 Thread guozhangwang
GitHub user guozhangwang opened a pull request:

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

HOTFIX: handle commit failed exception on stream thread's suspend task

1. Capture `CommitFailedException` in `StreamThread#suspendTasksAndState`; 
also fix log4j outputs for error and warn.

2. Minor fixes on StreamTask class for comments alignment etc.

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

$ git pull https://github.com/guozhangwang/kafka 
KHotfix-handle-commit-failed-exception-in-suspend

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

https://github.com/apache/kafka/pull/3574.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 #3574


commit 2d45430191c3dc417992c08454f9c550c1e6bb93
Author: Guozhang Wang 
Date:   2017-07-25T22:43:18Z

handle commit failed exception on stream thread




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[jira] [Created] (KAFKA-5640) Look into making acks=all the default setting.

2017-07-25 Thread Apurva Mehta (JIRA)
Apurva Mehta created KAFKA-5640:
---

 Summary: Look into making acks=all the default setting. 
 Key: KAFKA-5640
 URL: https://issues.apache.org/jira/browse/KAFKA-5640
 Project: Kafka
  Issue Type: Sub-task
Reporter: Apurva Mehta
Assignee: Apurva Mehta
 Fix For: 1.0.0


KAFKA-5494 proposed dropping the requirement for 
{{max.inflight.requests.per.connection=1}} for the idempotent producer. 

That is a stepping stone to enabling the idempotent producer by default without 
sacrificing performance.

A further step would be making {{acks=all}} the default setting as well. Then, 
with {{enable.idempotence=true}}, {{max.inflight.requests.per.connection=5}}, 
{{acks=all}}, {{retries=MAX_INT}}, we would have exactly once semantics with 
strong durability guarantees. 

This particular ticket is about investigating the performance degradation 
caused by {{acks=all}}. How much does throughput degrade? If it is significant, 
are there low hanging fruit in terms of code or config changes which would 
allow us to bridge most of the gap?




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


[GitHub] kafka pull request #3575: KAFKA-5634; Do not allow segment deletion beyond h...

2017-07-25 Thread hachikuji
GitHub user hachikuji opened a pull request:

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

KAFKA-5634; Do not allow segment deletion beyond high watermark



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

$ git pull https://github.com/hachikuji/kafka KAFKA-5634

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

https://github.com/apache/kafka/pull/3575.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 #3575


commit 28ba15871d954fa7036b26ebebc5268fbaf1b818
Author: Jason Gustafson 
Date:   2017-07-26T00:13:43Z

KAFKA-5634; Do not allow segment deletion beyond high watermark




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] kafka pull request #3565: KAFKA-5627; Reduce classes needed for LeaderAndIsr...

2017-07-25 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


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

2017-07-25 Thread Apache Jenkins Server
See 

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

Cloning the remote Git repository
Cloning repository https://git-wip-us.apache.org/repos/asf/kafka.git
 > git init  # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error cloning remote repo 'origin'
hudson.plugins.git.GitException: Command "git fetch --tags --progress 
https://git-wip-us.apache.org/repos/asf/kafka.git 
+refs/heads/*:refs/remotes/origin/*" returned status code 128:
stdout: 
stderr: fatal: unable to access 
'https://git-wip-us.apache.org/repos/asf/kafka.git/': Could not resolve host: 
git-wip-us.apache.org

at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandIn(CliGitAPIImpl.java:1903)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandWithCredentials(CliGitAPIImpl.java:1622)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.access$300(CliGitAPIImpl.java:71)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl$1.execute(CliGitAPIImpl.java:348)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl$2.execute(CliGitAPIImpl.java:545)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:153)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:146)
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:748)
at ..remote call to cassandra12(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 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.execute(RemoteGitImpl.java:146)
at sun.reflect.GeneratedMethodAccessor864.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.invoke(RemoteGitImpl.java:132)
at com.sun.proxy.$Proxy142.execute(Unknown Source)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1070)
at hudson.plugins.git.GitSCM.checkout(GitSCM.java:1110)
at hudson.scm.SCM.checkout(SCM.java:495)
at hudson.model.AbstractProject.checkout(AbstractProject.java:1276)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.defaultCheckout(AbstractBuild.java:560)
at jenkins.scm.SCMCheckoutStrategy.checkout(SCMCheckoutStrategy.java:86)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.run(AbstractBuild.java:485)
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)
ERROR: Error cloning remote repo 'origin'
Retrying after 10 seconds
 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM.checkout(GitSCM.java:1110)
at hudson.scm.SCM.checkout(SCM.java:495)
at hudson.model.AbstractProject.checkout(AbstractProject.java:1276)
at 
hudson.model.AbstractBuild$

[jira] [Resolved] (KAFKA-5627) Reduce classes needed for LeaderAndIsrPartitionState and MetadataPartitionState

2017-07-25 Thread Jiangjie Qin (JIRA)

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

Jiangjie Qin resolved KAFKA-5627.
-
Resolution: Fixed

> Reduce classes needed for LeaderAndIsrPartitionState and 
> MetadataPartitionState
> ---
>
> Key: KAFKA-5627
> URL: https://issues.apache.org/jira/browse/KAFKA-5627
> Project: Kafka
>  Issue Type: Improvement
>Reporter: Dong Lin
>Assignee: Dong Lin
>
> It will be cleaner to replace LeaderAndIsrPartitionState and 
> MetadataPartitionState in LeaderAndIsr.scala with 
> org.apache.kafka.common.requests.PartitionState and
> UpdateMetadataRequest.PartitionState respectively.



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


Re: [DISCUSS] KIP-174 - Deprecate and remove internal converter configs in WorkerConfig

2017-07-25 Thread Ewen Cheslack-Postava
Umesh,

Thanks for the KIP. Straightforward and I think it's a good change.
Unfortunately it is hard to tell how many people it would affect since we
can't tell how many people have adjusted that config, but I think this is
the right thing to do long term.

A couple of quick things that might be helpful to refine:

* Note that there are also some references in the example configs that we
should remove.
* It's nice to be explicit about when the removal is planned. This lets us
set expectations with users for timeframe (especially now that we have time
based releases), allows us to give info about the removal timeframe in log
error messages, and lets us file a JIRA against that release so we remember
to follow up. Given the update to 1.0.0 for the next release, we may also
need to adjust how we deal with deprecations/removal if we don't want to
have to wait all the way until 2.0 to remove (though it is unclear how
exactly we will be handling version bumps from now on).
* Migration path -- I think this is the major missing gap in the KIP. Do we
need a migration path? If not, presumably it is because people aren't using
any other converters in practice. Do we have some way of validating this (
https://issues.apache.org/jira/browse/KAFKA-3988 might be pretty convincing
evidence)? If there are some users using other converters, how would they
migrate to newer versions which would no longer support that?

-Ewen


On Fri, Jul 14, 2017 at 2:37 AM, UMESH CHAUDHARY 
wrote:

> Hi there,
> Resending as probably missed earlier to grab your attention.
>
> Regards,
> Umesh
>
> -- Forwarded message -
> From: UMESH CHAUDHARY 
> Date: Mon, 3 Jul 2017 at 11:04
> Subject: [DISCUSS] KIP-174 - Deprecate and remove internal converter
> configs in WorkerConfig
> To: dev@kafka.apache.org 
>
>
> Hello All,
> I have added a KIP recently to deprecate and remove internal converter
> configs in WorkerConfig.java class because these have ultimately just
> caused a lot more trouble and confusion than it is worth.
>
> Please find the KIP here
>  174+-+Deprecate+and+remove+internal+converter+configs+in+WorkerConfig>
> and
> the related JIRA here .
>
> Appreciate your review and comments.
>
> Regards,
> Umesh
>


Re: consumer group offset lag

2017-07-25 Thread Abhimanyu Nagrath
I am also facing the same issue.

On Tue, Jul 25, 2017 at 9:58 PM, Tarun Garg  wrote:

> I am sending this mail to DEV because i think this is a bug in AdminClient
> while getting the group offset.
>
> Hi,
>
> I run through a issue.
> i am trying to get the status of a consumer group.
>
> for this i have consumer and producer with topic named test_7 with
> partition count 1. It is working fine.
>
> 1. then i stoped my services and increased the number of partition to to
> and then check the status it doesn’t showed me the new partitions.
> 2. i started my producer to populated data still the consumer group status
> check gives me the same response.
> 3. finally i started my consumer and then i am able to see the data and lag
>
> This is not giving me the correct lag.
>
> Any idea how to get that information out.
>
> Thanks
> Tarun
>
> tarun.garg:tgarg kafka_2.12-0.10.2.0> bin/kafka-consumer-groups.sh
> --bootstrap-server localhost:9092 --describe --group ReactiveConsumerTest_3
> Note: This will only show information about consumers that use the Java
> consumer API (non-ZooKeeper-based consumers).
>
> Consumer group 'ReactiveConsumerTest_3' has no active members.
>
> TOPIC  PARTITION  CURRENT-OFFSET  LOG-END-OFFSET
> LAGCONSUMER-ID   HOST
>  CLIENT-ID
> test_7 0  113720  114324
> 604- -
> -
>
> tarun.garg:tgarg kafka_2.12-0.10.2.0> bin/kafka-consumer-groups.sh
> --bootstrap-server localhost:9092 --describe --group ReactiveConsumerTest_3
> Note: This will only show information about consumers that use the Java
> consumer API (non-ZooKeeper-based consumers).
>
>
> TOPIC  PARTITION  CURRENT-OFFSET  LOG-END-OFFSET
> LAGCONSUMER-ID   HOST
>  CLIENT-ID
> test_7 0  114700  114744
> 44 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 1  245 256
>  11 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 2  980 1024
> 44 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 3  14701536
> 66 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 4  245 256
>  11 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 5  12251280
> 55 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 6  490 512
>  22 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 7  17151792
> 77 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 8  17151792
> 77 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
> test_7 9  735 768
>  33 consumer-1-47b3f36-f897-460d-b178-d03f7f11a749   /172.17.0.1
>   consumer-1
>
>


Re: [VOTE] KIP-164 Add unavailablePartitionCount and per-partition Unavailable metrics

2017-07-25 Thread Joel Koshy
+1

On Thu, Jul 20, 2017 at 10:30 AM, Becket Qin  wrote:

> +1, Thanks for the KIP.
>
> On Thu, Jul 20, 2017 at 7:08 AM, Ismael Juma  wrote:
>
> > Thanks for the KIP, +1 (binding).
> >
> > On Thu, Jun 1, 2017 at 9:44 AM, Dong Lin  wrote:
> >
> > > Hi all,
> > >
> > > Can you please vote for KIP-164? The KIP can be found at
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-164-+Add+
> > > UnderMinIsrPartitionCount+and+per-partition+UnderMinIsr+metrics
> > > .
> > >
> > > Thanks,
> > > Dong
> > >
> >
>


Re: [DISCUSS] KIP-169 Lag-Aware Partition Assignment Strategy

2017-07-25 Thread Ewen Cheslack-Postava
Thanks for the KIP Grant.

First, to Vahid's feedback, I think lag is a pretty reasonable heuristic,
despite all those other factors. In normal cases I wouldn't expect network
latency, per-message processing time, and consumer configuration to vary
much within the group. Per-message processing time is the one I'd expect to
vary the most, but I would think that in the common case, the distribution
of processing times would be pretty reasonably similar.

I also had a few notes on the KIP:

* Motivation -- with PartitionAssignors, because they are already pluggable
without being added to AK, usually what I'd be looking for in a KIP's
motivation section is that this would be commonly used because it covers a
use case where the Range and RoundRobin assignors don't work well and
therefore it makes sense to include and maintain it as part of the core AK
project. I think the motivation here is that if you have particularly bad
imbalance (e.g. let's say you update your app to consume from an additional
topic, use offset reset earliest, and all the topic partitions from that
topic get assigned to the same consumer such that it never manages to catch
up on any of them). It makes sense, but the cases I can come up with where
this is a problem would generally be addressed by RoundRobinAssignor. Is
this something you're hitting regularly or have seen common requests for?
* Since the assignor only runs on rebalance, it cannot be reactive to
changing lag. I assume the motivating use case doesn't require it to be
dynamic, but only to handle a "catch up" use case?
* You mention the case with 0 lag looking like RangeAssignor
> (in this case the resulting assignment will be similar to that of the
RangeAssignor)
I think we would want the default to be similar to round robin.
RangeAssignor has imbalance problems.
* In the prototype, you implement Configurable as well as
PartitionAssignor. This means this wouldn't work generally unless we also
extended PartitionAssignor to implement Configurable since you can't just
set the configuration option.
* In step 2 of the algorithm, shouldn't we just process all topic
partitions together rather than working topic by topic?
* This is a greedy solution, it might be nice to say if there are any
guarantees about how close we are to optimal.


-Ewen

On Thu, Jul 13, 2017 at 1:49 PM, Vahid S Hashemian <
vahidhashem...@us.ibm.com> wrote:

> Hi Grant,
>
> Thank you for the KIP. Very well written and easy to understand.
>
> One question I have after reading the KIP: What are we targeting by using
> a Lag Aware assignment assignor?
>
> Is the goal to speed up consuming all messages from a topic?
> If that is the case, it sounds to me that assigning partitions based on
> only lag information would not be enough.
> There are other factors, like network latency, how fast a consumer is
> processing data, and consumer configuration (such as fetch.max.bytes,
> max.partition.fetch.bytes, ...) that impact how fast a consumer is able to
> consume messages.
>
> For example, let's say we have a topic with 4 partitions, and the lags are
> 1000, 100, 10, 1 for partitions 0 to 3.
> If we have two consumers c1 and c2 in the group, the Lag Aware assignment
> will be
> - c1: p0, p3 (total lag of 1001)
> - c2: p1, p2 (total lag of 110)
> Now if the speed c1 is consuming is 10% of the speed c2 is consuming then
> the opposite assignment (c1: p1, p2 - c2: p0, p3) would be more
> reasonable.
>
> I hope I'm not missing something in the KIP, and sorry if I misunderstood
> the purpose.
>
> Thanks.
> --Vahid
>
>
>
>
> From:   Grant Neale 
> To: "dev@kafka.apache.org" 
> Date:   06/18/2017 11:04 AM
> Subject:[DISCUSS] KIP-169 Lag-Aware Partition Assignment Strategy
>
>
>
> Hi all,
>
> I have raised a new KIP at
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 169+-+Lag-Aware+Partition+Assignment+Strategy
>
>
> The corresponding JIRA is at
> https://issues.apache.org/jira/browse/KAFKA-5337
>
> I look forward to your feedback.
>
> Regards,
> Grant Neale
>
>
>
>
>


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

2017-07-25 Thread Apache Jenkins Server
See 

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on cassandra12 (cassandra 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://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM.checkout(GitSCM.java:1110)
at hudson.scm.SCM.checkout(SCM.java:495)
at hudson.model.AbstractProject.checkout(AbstractProject.java:1276)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.defaultCheckout(AbstractBuild.java:560)
at jenkins.scm.SCMCheckoutStrategy.checkout(SCMCheckoutStrategy.java:86)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.run(AbstractBuild.java:485)
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)
Caused by: hudson.plugins.git.GitException: Command "git fetch --tags 
--progress https://git-wip-us.apache.org/repos/asf/kafka.git 
+refs/heads/*:refs/remotes/origin/*" returned status code 128:
stdout: 
stderr: fatal: unable to access 
'https://git-wip-us.apache.org/repos/asf/kafka.git/': Could not resolve host: 
git-wip-us.apache.org

at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandIn(CliGitAPIImpl.java:1903)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandWithCredentials(CliGitAPIImpl.java:1622)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.access$300(CliGitAPIImpl.java:71)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl$1.execute(CliGitAPIImpl.java:348)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:153)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:146)
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:748)
at ..remote call to cassandra12(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 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.execute(RemoteGitImpl.java:146)
at sun.reflect.GeneratedMethodAccessor864.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.invoke(RemoteGitImpl.java:132)
at com.sun.proxy.$Proxy104.execute(Unknown Source)
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:810)
... 11 more
ERROR: Error fetching remote repo 'origin'
Retrying after 10 seconds
 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM

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

2017-07-25 Thread Apache Jenkins Server
See 

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on cassandra12 (cassandra 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://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM.checkout(GitSCM.java:1110)
at hudson.scm.SCM.checkout(SCM.java:495)
at hudson.model.AbstractProject.checkout(AbstractProject.java:1276)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.defaultCheckout(AbstractBuild.java:560)
at jenkins.scm.SCMCheckoutStrategy.checkout(SCMCheckoutStrategy.java:86)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.run(AbstractBuild.java:485)
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)
Caused by: hudson.plugins.git.GitException: Command "git fetch --tags 
--progress https://git-wip-us.apache.org/repos/asf/kafka.git 
+refs/heads/*:refs/remotes/origin/*" returned status code 128:
stdout: 
stderr: fatal: unable to access 
'https://git-wip-us.apache.org/repos/asf/kafka.git/': Could not resolve host: 
git-wip-us.apache.org

at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandIn(CliGitAPIImpl.java:1903)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandWithCredentials(CliGitAPIImpl.java:1622)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.access$300(CliGitAPIImpl.java:71)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl$1.execute(CliGitAPIImpl.java:348)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:153)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:146)
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:748)
at ..remote call to cassandra12(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 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.execute(RemoteGitImpl.java:146)
at sun.reflect.GeneratedMethodAccessor864.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.invoke(RemoteGitImpl.java:132)
at com.sun.proxy.$Proxy104.execute(Unknown Source)
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:810)
... 11 more
ERROR: Error fetching remote repo 'origin'
Retrying after 10 seconds
 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM

[jira] [Created] (KAFKA-5641) Metadata request should be allowed to send no regardless of value for max.in.flight.requests.per.connection

2017-07-25 Thread huxihx (JIRA)
huxihx created KAFKA-5641:
-

 Summary: Metadata request should be allowed to send no regardless 
of value for max.in.flight.requests.per.connection
 Key: KAFKA-5641
 URL: https://issues.apache.org/jira/browse/KAFKA-5641
 Project: Kafka
  Issue Type: Improvement
  Components: producer 
Affects Versions: 0.11.0.0
Reporter: huxihx


Metadata request might not be able to be sent when 
`max.in.flight.requests.per.connection` is set to 1 and there is already an 
inflight request in the same node's queue, as show below:
{code:title=NetworkClient.java|borderStyle=solid}
private long maybeUpdate(long now, Node node) {
String nodeConnectionId = node.idString();
if (canSendRequest(nodeConnectionId)) {
..
}
{code}

However, setting `max.in.flight.requests.per.connection` to 1 actually means no 
out-of-order for the produced records, Metadata requests should have no related 
with this config. We don't have to check the inflight request's queue size when 
sending Metadata request.

[~ijuma] Does it make any sense? If yes, I will work on it.



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


[GitHub] kafka pull request #2330: KAFKA-4602 - KIP-72 - Allow putting a bound on mem...

2017-07-25 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


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

2017-07-25 Thread Apache Jenkins Server
See 

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on cassandra12 (cassandra 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://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM.checkout(GitSCM.java:1110)
at hudson.scm.SCM.checkout(SCM.java:495)
at hudson.model.AbstractProject.checkout(AbstractProject.java:1276)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.defaultCheckout(AbstractBuild.java:560)
at jenkins.scm.SCMCheckoutStrategy.checkout(SCMCheckoutStrategy.java:86)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.run(AbstractBuild.java:485)
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)
Caused by: hudson.plugins.git.GitException: Command "git fetch --tags 
--progress https://git-wip-us.apache.org/repos/asf/kafka.git 
+refs/heads/*:refs/remotes/origin/*" returned status code 128:
stdout: 
stderr: fatal: unable to access 
'https://git-wip-us.apache.org/repos/asf/kafka.git/': Could not resolve host: 
git-wip-us.apache.org

at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandIn(CliGitAPIImpl.java:1903)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandWithCredentials(CliGitAPIImpl.java:1622)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.access$300(CliGitAPIImpl.java:71)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl$1.execute(CliGitAPIImpl.java:348)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:153)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:146)
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:748)
at ..remote call to cassandra12(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 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.execute(RemoteGitImpl.java:146)
at sun.reflect.GeneratedMethodAccessor864.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.invoke(RemoteGitImpl.java:132)
at com.sun.proxy.$Proxy104.execute(Unknown Source)
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:810)
... 11 more
ERROR: Error fetching remote repo 'origin'
Retrying after 10 seconds
 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM

[jira] [Resolved] (KAFKA-5501) use async zookeeper apis everywhere

2017-07-25 Thread Jun Rao (JIRA)

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

Jun Rao resolved KAFKA-5501.

   Resolution: Fixed
Fix Version/s: 1.0.0

Issue resolved by pull request 3427
[https://github.com/apache/kafka/pull/3427]

> use async zookeeper apis everywhere
> ---
>
> Key: KAFKA-5501
> URL: https://issues.apache.org/jira/browse/KAFKA-5501
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Onur Karaman
>Assignee: Onur Karaman
> Fix For: 1.0.0
>
>
> Synchronous zookeeper writes means that we wait an entire round trip before 
> doing the next write. These synchronous writes are happening at a 
> per-partition granularity in several places, so partition-heavy clusters 
> suffer from the controller doing many sequential round trips to zookeeper.
> * PartitionStateMachine.electLeaderForPartition updates leaderAndIsr in 
> zookeeper on transition to OnlinePartition. This gets triggered per-partition 
> sequentially with synchronous writes during controlled shutdown of the 
> shutting down broker's replicas for which it is the leader.
> * ReplicaStateMachine updates leaderAndIsr in zookeeper on transition to 
> OfflineReplica when calling KafkaController.removeReplicaFromIsr. This gets 
> triggered per-partition sequentially with synchronous writes for failed or 
> controlled shutdown brokers.



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


[GitHub] kafka pull request #3427: KAFKA-5501 [WIP]: use async zookeeper apis everywh...

2017-07-25 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


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

2017-07-25 Thread Apache Jenkins Server
See 

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on cassandra12 (cassandra 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://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM.checkout(GitSCM.java:1110)
at hudson.scm.SCM.checkout(SCM.java:495)
at hudson.model.AbstractProject.checkout(AbstractProject.java:1276)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.defaultCheckout(AbstractBuild.java:560)
at jenkins.scm.SCMCheckoutStrategy.checkout(SCMCheckoutStrategy.java:86)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.run(AbstractBuild.java:485)
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)
Caused by: hudson.plugins.git.GitException: Command "git fetch --tags 
--progress https://git-wip-us.apache.org/repos/asf/kafka.git 
+refs/heads/*:refs/remotes/origin/*" returned status code 128:
stdout: 
stderr: fatal: unable to access 
'https://git-wip-us.apache.org/repos/asf/kafka.git/': Could not resolve host: 
git-wip-us.apache.org

at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandIn(CliGitAPIImpl.java:1903)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandWithCredentials(CliGitAPIImpl.java:1622)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.access$300(CliGitAPIImpl.java:71)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl$1.execute(CliGitAPIImpl.java:348)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:153)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:146)
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:748)
at ..remote call to cassandra12(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 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.execute(RemoteGitImpl.java:146)
at sun.reflect.GeneratedMethodAccessor864.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.invoke(RemoteGitImpl.java:132)
at com.sun.proxy.$Proxy104.execute(Unknown Source)
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:810)
... 11 more
ERROR: Error fetching remote repo 'origin'
Retrying after 10 seconds
 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM

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

2017-07-25 Thread Apache Jenkins Server
See 

--
Started by an SCM change
[EnvInject] - Loading node environment variables.
Building remotely on cassandra12 (cassandra 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://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM.checkout(GitSCM.java:1110)
at hudson.scm.SCM.checkout(SCM.java:495)
at hudson.model.AbstractProject.checkout(AbstractProject.java:1276)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.defaultCheckout(AbstractBuild.java:560)
at jenkins.scm.SCMCheckoutStrategy.checkout(SCMCheckoutStrategy.java:86)
at 
hudson.model.AbstractBuild$AbstractBuildExecution.run(AbstractBuild.java:485)
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)
Caused by: hudson.plugins.git.GitException: Command "git fetch --tags 
--progress https://git-wip-us.apache.org/repos/asf/kafka.git 
+refs/heads/*:refs/remotes/origin/*" returned status code 128:
stdout: 
stderr: fatal: unable to access 
'https://git-wip-us.apache.org/repos/asf/kafka.git/': Could not resolve host: 
git-wip-us.apache.org

at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandIn(CliGitAPIImpl.java:1903)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.launchCommandWithCredentials(CliGitAPIImpl.java:1622)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl.access$300(CliGitAPIImpl.java:71)
at 
org.jenkinsci.plugins.gitclient.CliGitAPIImpl$1.execute(CliGitAPIImpl.java:348)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:153)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler$1.call(RemoteGitImpl.java:146)
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:748)
at ..remote call to cassandra12(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 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.execute(RemoteGitImpl.java:146)
at sun.reflect.GeneratedMethodAccessor864.invoke(Unknown Source)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at 
org.jenkinsci.plugins.gitclient.RemoteGitImpl$CommandInvocationHandler.invoke(RemoteGitImpl.java:132)
at com.sun.proxy.$Proxy104.execute(Unknown Source)
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:810)
... 11 more
ERROR: Error fetching remote repo 'origin'
Retrying after 10 seconds
 > git rev-parse --is-inside-work-tree # timeout=10
Fetching changes from the remote Git repository
 > git config remote.origin.url 
 > https://git-wip-us.apache.org/repos/asf/kafka.git # timeout=10
Fetching upstream changes from https://git-wip-us.apache.org/repos/asf/kafka.git
 > git --version # timeout=10
 > git fetch --tags --progress 
 > https://git-wip-us.apache.org/repos/asf/kafka.git 
 > +refs/heads/*:refs/remotes/origin/*
ERROR: Error fetching remote repo 'origin'
hudson.plugins.git.GitException: Failed to fetch from 
https://git-wip-us.apache.org/repos/asf/kafka.git
at hudson.plugins.git.GitSCM.fetchFrom(GitSCM.java:812)
at hudson.plugins.git.GitSCM.retrieveChanges(GitSCM.java:1079)
at hudson.plugins.git.GitSCM

[jira] [Reopened] (KAFKA-5501) use async zookeeper apis everywhere

2017-07-25 Thread Ismael Juma (JIRA)

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

Ismael Juma reopened KAFKA-5501:


> use async zookeeper apis everywhere
> ---
>
> Key: KAFKA-5501
> URL: https://issues.apache.org/jira/browse/KAFKA-5501
> Project: Kafka
>  Issue Type: Sub-task
>Reporter: Onur Karaman
>Assignee: Onur Karaman
> Fix For: 1.0.0
>
>
> Synchronous zookeeper writes means that we wait an entire round trip before 
> doing the next write. These synchronous writes are happening at a 
> per-partition granularity in several places, so partition-heavy clusters 
> suffer from the controller doing many sequential round trips to zookeeper.
> * PartitionStateMachine.electLeaderForPartition updates leaderAndIsr in 
> zookeeper on transition to OnlinePartition. This gets triggered per-partition 
> sequentially with synchronous writes during controlled shutdown of the 
> shutting down broker's replicas for which it is the leader.
> * ReplicaStateMachine updates leaderAndIsr in zookeeper on transition to 
> OfflineReplica when calling KafkaController.removeReplicaFromIsr. This gets 
> triggered per-partition sequentially with synchronous writes for failed or 
> controlled shutdown brokers.



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