Re: [DISCUSS] KIP-487: Automatic Topic Creation on Producer

2019-08-05 Thread Ismael Juma
Harsha,

Rogue clients can use the admin client to create topics and partitions.
ACLs and policies can help in that case as well as this one.

Ismael

On Mon, Aug 5, 2019, 7:48 PM Harsha Chintalapani  wrote:

> Hi Justine,
>  Thanks for the KIP.
> "When server-side auto-creation is disabled, client-side auto-creation will
> try to use client-side configurations"
> If I understand correctly, this KIP is removing any server-side blocking
> client auto creation of topic?
> if so this will present potential issue of rogue client creating ton of
> topic-partitions and potentially bringing down the service for everyone or
> degrade the service itself.
> By reading the KIP its not clear to me that there is a clear way to block
> auto creation topics of all together from clients by server side config.
> Server side configs of default topic, partitions should take higher
> precedence and client shouldn't be able to create a topic with higher no.of
> partitions, replication than what server config specifies.
>
> Thanks,
> Harsha
>
>
>
> On Mon, Aug 05, 2019 at 5:24 PM, Justine Olshan 
> wrote:
>
> > Hi all,
> > I made some changes to the KIP. Hopefully this configuration change will
> > make things a little clearer.
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/
> > KIP-487%3A+Client-side+Automatic+Topic+Creation+on+Producer
> >
> > Please let me know if you have any feedback or questions!
> >
> > Thank you,
> > Justine
> >
> > On Wed, Jul 31, 2019 at 1:44 PM Colin McCabe  wrote:
> >
> > Hi Mickael,
> >
> > I think you bring up a good point. It would be better if we didn't ever
> > have to set up client-side configuration for this feature, and KIP-464
> > would let us skip this entirely.
> >
> > On Wed, Jul 31, 2019, at 09:19, Justine Olshan wrote:
> >
> > Hi Mickael,
> > I agree that KIP-464 works on newer brokers, but I was a bit worried how
> > things would play out on older brokers that* do not *have KIP 464
> >
> > included.
> >
> > Is it enough to throw an error in this case when producer configs are not
> > specified?
> >
> > I think the right thing to do would be to log an error message in the
> > client. We will need to have that capability in any case, to cover
> > scenarios like the client trying to auto-create a topic that they don't
> > have permission to create. Or a client trying to create a topic on a
> broker
> > so old that CreateTopicsRequest is not supported.
> >
> > The big downside to relying on KIP-464 is that it is a very recent
> feature
> > -- so recent that it hasn't even made its way to any official Apache
> > release. It's scheduled for the upcoming 2.4 release in a few months.
> >
> > So if you view this KIP as a step towards removing broker-side
> > auto-create, you might want to support older brokers just to accelerate
> > adoption, and hasten the day when we can finally flip broker-side
> > auto-create to off (or even remove it entirely).
> >
> > I have to agree, though, that having client-side configurations for
> number
> > of partitions and replication factor is messy. Maybe it would be worth it
> > to restrict support to post-KIP-464 brokers, if we could avoid creating
> > more configs.
> >
> > best,
> > Colin
> >
> > On Wed, Jul 31, 2019 at 9:10 AM Mickael Maison  >
> > wrote:
> >
> > Hi Justine,
> >
> > We can rely on KIP-464 which allows to omit the partition count or
> > replication factor when creating a topic. In that case, the broker
> defaults
> > are used.
> >
> > On Wed, Jul 31, 2019 at 4:55 PM Justine Olshan 
> > wrote:
> >
> > Michael,
> > That makes sense to me!
> > To clarify, in the current state of the KIP, the producer does not
> >
> > rely
> >
> > on
> >
> > the broker to autocreate--if the broker's config is disabled, then
> >
> > the
> >
> > producer can autocreate on its own with a create topic request (the
> >
> > same
> >
> > type of request the admin client uses).
> > However, if both configs are enabled, the broker will autocreate
> >
> > through
> >
> > a
> >
> > metadata request before the producer gets a chance. Of course, the way to
> > avoid this, is to do as you suggested, and set
> >
> > the
> >
> > "allow_auto_topic_creation" field to false.
> >
> > I think the only thing we need to be careful with in this setup is
> >
> > without
> >
> > KIP 464, we can not use broker defaults for this topic. A user needs
> >
> > to
> >
> > specify the number of partition and replication factor in the config. An
> > alternative to this is to have coded defaults for when these
> >
> > configs
> >
> > are
> >
> > unspecified, but it is not immediately apparent what these defaults
> >
> > should
> >
> > be.
> >
> > Thanks again for reading my KIP,
> > Justine
> >
> > On Wed, Jul 31, 2019 at 4:19 AM Mickael Maison <
> >
> > mickael.mai...@gmail.com
> >
> > wrote:
> >
> > Hi Justine,
> >
> > Thanks for the response!
> > In my opinion, it would be better if the producer did not rely at
> >
> > all
> >
> > on the broker auto create feature as 

Re: [DISCUSS] KIP-487: Automatic Topic Creation on Producer

2019-08-05 Thread Harsha Chintalapani
Hi Justine,
 Thanks for the KIP.
"When server-side auto-creation is disabled, client-side auto-creation will
try to use client-side configurations"
If I understand correctly, this KIP is removing any server-side blocking
client auto creation of topic?
if so this will present potential issue of rogue client creating ton of
topic-partitions and potentially bringing down the service for everyone or
degrade the service itself.
By reading the KIP its not clear to me that there is a clear way to block
auto creation topics of all together from clients by server side config.
Server side configs of default topic, partitions should take higher
precedence and client shouldn't be able to create a topic with higher no.of
partitions, replication than what server config specifies.

Thanks,
Harsha



On Mon, Aug 05, 2019 at 5:24 PM, Justine Olshan 
wrote:

> Hi all,
> I made some changes to the KIP. Hopefully this configuration change will
> make things a little clearer.
>
> https://cwiki.apache.org/confluence/display/KAFKA/
> KIP-487%3A+Client-side+Automatic+Topic+Creation+on+Producer
>
> Please let me know if you have any feedback or questions!
>
> Thank you,
> Justine
>
> On Wed, Jul 31, 2019 at 1:44 PM Colin McCabe  wrote:
>
> Hi Mickael,
>
> I think you bring up a good point. It would be better if we didn't ever
> have to set up client-side configuration for this feature, and KIP-464
> would let us skip this entirely.
>
> On Wed, Jul 31, 2019, at 09:19, Justine Olshan wrote:
>
> Hi Mickael,
> I agree that KIP-464 works on newer brokers, but I was a bit worried how
> things would play out on older brokers that* do not *have KIP 464
>
> included.
>
> Is it enough to throw an error in this case when producer configs are not
> specified?
>
> I think the right thing to do would be to log an error message in the
> client. We will need to have that capability in any case, to cover
> scenarios like the client trying to auto-create a topic that they don't
> have permission to create. Or a client trying to create a topic on a broker
> so old that CreateTopicsRequest is not supported.
>
> The big downside to relying on KIP-464 is that it is a very recent feature
> -- so recent that it hasn't even made its way to any official Apache
> release. It's scheduled for the upcoming 2.4 release in a few months.
>
> So if you view this KIP as a step towards removing broker-side
> auto-create, you might want to support older brokers just to accelerate
> adoption, and hasten the day when we can finally flip broker-side
> auto-create to off (or even remove it entirely).
>
> I have to agree, though, that having client-side configurations for number
> of partitions and replication factor is messy. Maybe it would be worth it
> to restrict support to post-KIP-464 brokers, if we could avoid creating
> more configs.
>
> best,
> Colin
>
> On Wed, Jul 31, 2019 at 9:10 AM Mickael Maison 
> wrote:
>
> Hi Justine,
>
> We can rely on KIP-464 which allows to omit the partition count or
> replication factor when creating a topic. In that case, the broker defaults
> are used.
>
> On Wed, Jul 31, 2019 at 4:55 PM Justine Olshan 
> wrote:
>
> Michael,
> That makes sense to me!
> To clarify, in the current state of the KIP, the producer does not
>
> rely
>
> on
>
> the broker to autocreate--if the broker's config is disabled, then
>
> the
>
> producer can autocreate on its own with a create topic request (the
>
> same
>
> type of request the admin client uses).
> However, if both configs are enabled, the broker will autocreate
>
> through
>
> a
>
> metadata request before the producer gets a chance. Of course, the way to
> avoid this, is to do as you suggested, and set
>
> the
>
> "allow_auto_topic_creation" field to false.
>
> I think the only thing we need to be careful with in this setup is
>
> without
>
> KIP 464, we can not use broker defaults for this topic. A user needs
>
> to
>
> specify the number of partition and replication factor in the config. An
> alternative to this is to have coded defaults for when these
>
> configs
>
> are
>
> unspecified, but it is not immediately apparent what these defaults
>
> should
>
> be.
>
> Thanks again for reading my KIP,
> Justine
>
> On Wed, Jul 31, 2019 at 4:19 AM Mickael Maison <
>
> mickael.mai...@gmail.com
>
> wrote:
>
> Hi Justine,
>
> Thanks for the response!
> In my opinion, it would be better if the producer did not rely at
>
> all
>
> on the broker auto create feature as this is what we're aiming to
> deprecate. When requesting metadata we can set the
> "allow_auto_topic_creation" field to false to avoid the broker auto
> creation. Then if the topic is not existing, send a CreateTopicRequest.
>
> What do you think?
>
> On Mon, Jul 29, 2019 at 6:34 PM Justine Olshan <
>
> jols...@confluent.io>
>
> wrote:
>
> Currently the way it is implemented, the broker auto-creation
>
> configuration
>
> takes precedence. The producer will not use the CreateTopics
>
> request.
>
> (Technically it can--but 

Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-08-05 Thread Boyang Chen
Yep, Guozhang I think that would be best as passing in an entire consumer
instance is indeed cumbersome.

Just saw you updated KIP-429, I will follow-up to change 447 as well.

Best,
Boyang

On Mon, Aug 5, 2019 at 3:18 PM Guozhang Wang  wrote:

> okay I think I understand your concerns about ConsumerGroupMetadata now: if
> we still want to only call initTxns once, then we should allow the whatever
> passed-in parameter to reflect the latest value of generation id whenever
> sending the offset fetch request.
>
> Whereas the current ConsumerGroupMetadata is a static object.
>
> Maybe we can consider having an extended class of ConsumerGroupMetadata
> whose values are updated from the consumer's rebalance callback?
>
>
> Guozhang
>
>
> On Mon, Aug 5, 2019 at 9:26 AM Boyang Chen 
> wrote:
>
> > Thank you Guozhang for the reply! I'm curious whether KIP-429 has
> reflected
> > the latest change on ConsumerGroupMetadata? Also regarding question one,
> > the group metadata needs to be accessed via callback, does that mean we
> > need a separate producer API such like
> > "producer.refreshMetadata(groupMetadata)" to be able to access it instead
> > of passing in the consumer instance?
> >
> > Boyang
> >
> > On Fri, Aug 2, 2019 at 4:36 PM Guozhang Wang  wrote:
> >
> > > Thanks Boyang,
> > >
> > > I've made another pass on KIP-447 as well as
> > > https://github.com/apache/kafka/pull/7078, and have some minor
> comments
> > > about the proposed API:
> > >
> > > 1. it seems instead of needing the whole KafkaConsumer object, you'd
> only
> > > need the "ConsumerGroupMetadata", in that case can we just pass in that
> > > object into the initTxns call?
> > >
> > > 2. the current trunk already has a public class named
> > > (ConsumerGroupMetadata)
> > > under o.a.k.clients.consumer created by KIP-429. If we want to just use
> > > that then maybe it makes less sense to declare a base GroupMetadata as
> we
> > > are already leaking such information on the assignor anyways.
> > >
> > >
> > > Guozhang
> > >
> > > On Tue, Jul 30, 2019 at 1:55 PM Boyang Chen <
> reluctanthero...@gmail.com>
> > > wrote:
> > >
> > > > Thank you Guozhang for the reply. We will consider the interface
> change
> > > > from 429 as a backup plan for 447.
> > > >
> > > > And bumping this thread for more discussion.
> > > >
> > > > On Mon, Jul 22, 2019 at 6:28 PM Guozhang Wang 
> > > wrote:
> > > >
> > > > > On Sat, Jul 20, 2019 at 9:50 AM Boyang Chen <
> > > reluctanthero...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Thank you Guozhang for the suggestion! I would normally prefer
> > > naming a
> > > > > > flag corresponding to its functionality. Seems to me
> > > `isolation_level`
> > > > > > makes us another hop on information track.
> > > > > >
> > > > > > Fair enough, let's use a separate flag name then :)
> > > > >
> > > > >
> > > > > > As for the generation.id exposure, I'm fine leveraging the new
> API
> > > > from
> > > > > > 429, but however is that design finalized yet, and whether the
> API
> > > will
> > > > > be
> > > > > > added on the generic Consumer interface?
> > > > > >
> > > > > > The current PartitionAssignor is inside `internals` package and
> in
> > > > > KIP-429
> > > > > we are going to create a new interface out of `internals` to really
> > > make
> > > > it
> > > > > public APIs, and as part of that we are refactoring some of its
> > method
> > > > > signatures. I just feel some of the newly introduced classes can be
> > > > reused
> > > > > in your KIP as well, i.e. just for code succinctness, but no
> > semantical
> > > > > indications.
> > > > >
> > > > >
> > > > > > Boyang
> > > > > >
> > > > > > On Fri, Jul 19, 2019 at 3:57 PM Guozhang Wang <
> wangg...@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Boyang, thanks for the updated proposal!
> > > > > > >
> > > > > > > 3.a. As Jason mentioned, with EOS enabled we still need to
> > augment
> > > > the
> > > > > > > offset fetch request with a boolean to indicate "give me an
> > > retriable
> > > > > > error
> > > > > > > code if there's pending offset, rather than sending me the
> > > committed
> > > > > > offset
> > > > > > > immediately". Personally I still feel it is okay to piggy-back
> on
> > > the
> > > > > > > ISOLATION_LEVEL boolean, but I'm also fine with another
> > > > > > `await_transaction`
> > > > > > > boolean if you feel strongly about it.
> > > > > > >
> > > > > > > 10. About the exposure of generation id, there may be some
> > > > refactoring
> > > > > > work
> > > > > > > coming from KIP-429 that can benefit KIP-447 as well since we
> are
> > > > > > wrapping
> > > > > > > the consumer subscription / assignment data in new classes.
> Note
> > > that
> > > > > > > current proposal does not `generationId` since with the
> > cooperative
> > > > > > sticky
> > > > > > > assignor we think it is not necessary for correctness, but also
> > if
> > > we
> > > > > > agree
> > > > > > > it is okay to expose it we can potentially include it in
> > > > > 

Re: [DISCUSS] KIP-487: Automatic Topic Creation on Producer

2019-08-05 Thread Justine Olshan
Hi all,
I made some changes to the KIP. Hopefully this configuration change will
make things a little clearer.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-487%3A+Client-side+Automatic+Topic+Creation+on+Producer

Please let me know if you have any feedback or questions!

Thank you,
Justine

On Wed, Jul 31, 2019 at 1:44 PM Colin McCabe  wrote:

> Hi Mickael,
>
> I think you bring up a good point.  It would be better if we didn't ever
> have to set up client-side configuration for this feature, and KIP-464
> would let us skip this entirely.
>
> On Wed, Jul 31, 2019, at 09:19, Justine Olshan wrote:
> > Hi Mickael,
> > I agree that KIP-464 works on newer brokers, but I was a bit worried how
> > things would play out on older brokers that* do not *have KIP 464
> included.
> > Is it enough to throw an error in this case when producer configs are not
> > specified?
>
> I think the right thing to do would be to log an error message in the
> client.  We will need to have that capability in any case, to cover
> scenarios like the client trying to auto-create a topic that they don't
> have permission to create.  Or a client trying to create a topic on a
> broker so old that CreateTopicsRequest is not supported.
>
> The big downside to relying on KIP-464 is that it is a very recent feature
> -- so recent that it hasn't even made its way to any official Apache
> release.  It's scheduled for the upcoming 2.4 release in a few months.
>
> So if you view this KIP as a step towards removing broker-side
> auto-create, you might want to support older brokers just to accelerate
> adoption, and hasten the day when we can finally flip broker-side
> auto-create to off (or even remove it entirely).
>
> I have to agree, though, that having client-side configurations for number
> of partitions and replication factor is messy.  Maybe it would be worth it
> to restrict support to post-KIP-464 brokers, if we could avoid creating
> more configs.
>
> best,
> Colin
>
>
> > On Wed, Jul 31, 2019 at 9:10 AM Mickael Maison  >
> > wrote:
> >
> > > Hi Justine,
> > >
> > > We can rely on KIP-464 which allows to omit the partition count or
> > > replication factor when creating a topic. In that case, the broker
> > > defaults are used.
> > >
> > > On Wed, Jul 31, 2019 at 4:55 PM Justine Olshan 
> > > wrote:
> > > >
> > > > Michael,
> > > > That makes sense to me!
> > > > To clarify, in the current state of the KIP, the producer does not
> rely
> > > on
> > > > the broker to autocreate--if the broker's config is disabled, then
> the
> > > > producer can autocreate on its own with a create topic request (the
> same
> > > > type of request the admin client uses).
> > > > However, if both configs are enabled, the broker will autocreate
> through
> > > a
> > > > metadata request before the producer gets a chance.
> > > > Of course, the way to avoid this, is to do as you suggested, and set
> the
> > > > "allow_auto_topic_creation" field to false.
> > > >
> > > > I think the only thing we need to be careful with in this setup is
> > > without
> > > > KIP 464, we can not use broker defaults for this topic. A user needs
> to
> > > > specify the number of partition and replication factor in the config.
> > > > An alternative to this is to have coded defaults for when these
> configs
> > > are
> > > > unspecified, but it is not immediately apparent what these defaults
> > > should
> > > > be.
> > > >
> > > > Thanks again for reading my KIP,
> > > > Justine
> > > >
> > > > On Wed, Jul 31, 2019 at 4:19 AM Mickael Maison <
> mickael.mai...@gmail.com
> > > >
> > > > wrote:
> > > >
> > > > > Hi Justine,
> > > > >
> > > > > Thanks for the response!
> > > > > In my opinion, it would be better if the producer did not rely at
> all
> > > > > on the broker auto create feature as this is what we're aiming to
> > > > > deprecate. When requesting metadata we can set the
> > > > > "allow_auto_topic_creation" field to false to avoid the broker auto
> > > > > creation. Then if the topic is not existing, send a
> > > > > CreateTopicRequest.
> > > > >
> > > > > What do you think?
> > > > >
> > > > > On Mon, Jul 29, 2019 at 6:34 PM Justine Olshan <
> jols...@confluent.io>
> > > > > wrote:
> > > > > >
> > > > > > Currently the way it is implemented, the broker auto-creation
> > > > > configuration
> > > > > > takes precedence. The producer will not use the CreateTopics
> request.
> > > > > > (Technically it can--but the topic will already be created
> through
> > > the
> > > > > > broker, so it will never try to create the topic.)
> > > > > > It is possible to change this however, and I'd be happy to
> discuss
> > > the
> > > > > > benefits of this alternative.
> > > > > >
> > > > > > Thank you,
> > > > > > Justine
> > > > > >
> > > > > > On Mon, Jul 29, 2019 at 10:26 AM Mickael Maison <
> > > > > mickael.mai...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Justine,
> > > > > > >
> > > > > > > Thanks for the KIP!
> > > > > > >
> > > > > > > In case auto 

[jira] [Created] (KAFKA-8757) Flaky Test SaslSslAdminClientIntegrationTest#testIncrementalAlterConfigsForLog4jLogLevelsCanResetLoggerToCurrentRoot

2019-08-05 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-8757:
--

 Summary: Flaky Test 
SaslSslAdminClientIntegrationTest#testIncrementalAlterConfigsForLog4jLogLevelsCanResetLoggerToCurrentRoot
 Key: KAFKA-8757
 URL: https://issues.apache.org/jira/browse/KAFKA-8757
 Project: Kafka
  Issue Type: Bug
  Components: admin, core, unit tests
Affects Versions: 2.4.0
Reporter: Matthias J. Sax


[https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/23942/testReport/junit/kafka.api/SaslSslAdminClientIntegrationTest/testIncrementalAlterConfigsForLog4jLogLevelsCanResetLoggerToCurrentRoot/]
{quote}org.junit.ComparisonFailure: expected:<[TRACE]> but was:<[INFO]> at 
org.junit.Assert.assertEquals(Assert.java:117) at 
org.junit.Assert.assertEquals(Assert.java:146) at 
kafka.api.AdminClientIntegrationTest.testIncrementalAlterConfigsForLog4jLogLevelsCanResetLoggerToCurrentRoot(AdminClientIntegrationTest.scala:1918){quote}
STDOUT
{quote}[2019-08-05 20:46:05,440] ERROR [KafkaApi-1] Error when handling 
request: clientId=adminclient-330, correlationId=4, api=CREATE_ACLS, 
body=\{creations=[{resource_type=2,resource_name=foobar,resource_pattern_type=3,principal=User:ANONYMOUS,host=*,operation=3,permission_type=3},\{resource_type=5,resource_name=transactional_id,resource_pattern_type=3,principal=User:ANONYMOUS,host=*,operation=4,permission_type=3}]}
 (kafka.server.KafkaApis:76) 
org.apache.kafka.common.errors.ClusterAuthorizationException: Request 
Request(processor=1, connectionId=127.0.0.1:42483-127.0.0.1:57808-0, 
session=Session(User:client2,localhost/127.0.0.1), 
listenerName=ListenerName(SASL_SSL), securityProtocol=SASL_SSL, buffer=null) is 
not authorized.{quote}
{quote}[2019-08-05 21:11:53,030] ERROR [ReplicaFetcher replicaId=1, leaderId=2, 
fetcherId=0] Error for partition topic-0 at offset 0 
(kafka.server.ReplicaFetcherThread:76) 
org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server 
does not host this topic-partition.{quote}



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (KAFKA-8756) Flaky Test SaslSslAdminClientIntegrationTest#testIncrementalAlterConfigsForLog4jLogLevels

2019-08-05 Thread Matthias J. Sax (JIRA)
Matthias J. Sax created KAFKA-8756:
--

 Summary: Flaky Test 
SaslSslAdminClientIntegrationTest#testIncrementalAlterConfigsForLog4jLogLevels
 Key: KAFKA-8756
 URL: https://issues.apache.org/jira/browse/KAFKA-8756
 Project: Kafka
  Issue Type: Bug
  Components: admin, core, unit tests
Affects Versions: 2.4.0
Reporter: Matthias J. Sax


[https://builds.apache.org/job/kafka-pr-jdk8-scala2.11/23942/testReport/junit/kafka.api/SaslSslAdminClientIntegrationTest/testIncrementalAlterConfigsForLog4jLogLevels/]
{quote}org.junit.ComparisonFailure: expected:<[OFF]> but was:<[INFO]> at 
org.junit.Assert.assertEquals(Assert.java:117) at 
org.junit.Assert.assertEquals(Assert.java:146) at 
kafka.api.AdminClientIntegrationTest.testIncrementalAlterConfigsForLog4jLogLevels(AdminClientIntegrationTest.scala:1850){quote}
STDOUT
{quote}[2019-08-05 20:46:05,440] ERROR [KafkaApi-1] Error when handling 
request: clientId=adminclient-330, correlationId=4, api=CREATE_ACLS, 
body=\{creations=[{resource_type=2,resource_name=foobar,resource_pattern_type=3,principal=User:ANONYMOUS,host=*,operation=3,permission_type=3},\{resource_type=5,resource_name=transactional_id,resource_pattern_type=3,principal=User:ANONYMOUS,host=*,operation=4,permission_type=3}]}
 (kafka.server.KafkaApis:76) 
org.apache.kafka.common.errors.ClusterAuthorizationException: Request 
Request(processor=1, connectionId=127.0.0.1:42483-127.0.0.1:57808-0, 
session=Session(User:client2,localhost/127.0.0.1), 
listenerName=ListenerName(SASL_SSL), securityProtocol=SASL_SSL, buffer=null) is 
not authorized.{quote}
{quote}[2019-08-05 21:14:35,816] ERROR [ReplicaFetcher replicaId=2, leaderId=1, 
fetcherId=0] Error for partition unclean-test-topic-1-0 at offset 0 
(kafka.server.ReplicaFetcherThread:76) 
org.apache.kafka.common.errors.UnknownTopicOrPartitionException: This server 
does not host this topic-partition.{quote}



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-08-05 Thread Guozhang Wang
okay I think I understand your concerns about ConsumerGroupMetadata now: if
we still want to only call initTxns once, then we should allow the whatever
passed-in parameter to reflect the latest value of generation id whenever
sending the offset fetch request.

Whereas the current ConsumerGroupMetadata is a static object.

Maybe we can consider having an extended class of ConsumerGroupMetadata
whose values are updated from the consumer's rebalance callback?


Guozhang


On Mon, Aug 5, 2019 at 9:26 AM Boyang Chen 
wrote:

> Thank you Guozhang for the reply! I'm curious whether KIP-429 has reflected
> the latest change on ConsumerGroupMetadata? Also regarding question one,
> the group metadata needs to be accessed via callback, does that mean we
> need a separate producer API such like
> "producer.refreshMetadata(groupMetadata)" to be able to access it instead
> of passing in the consumer instance?
>
> Boyang
>
> On Fri, Aug 2, 2019 at 4:36 PM Guozhang Wang  wrote:
>
> > Thanks Boyang,
> >
> > I've made another pass on KIP-447 as well as
> > https://github.com/apache/kafka/pull/7078, and have some minor comments
> > about the proposed API:
> >
> > 1. it seems instead of needing the whole KafkaConsumer object, you'd only
> > need the "ConsumerGroupMetadata", in that case can we just pass in that
> > object into the initTxns call?
> >
> > 2. the current trunk already has a public class named
> > (ConsumerGroupMetadata)
> > under o.a.k.clients.consumer created by KIP-429. If we want to just use
> > that then maybe it makes less sense to declare a base GroupMetadata as we
> > are already leaking such information on the assignor anyways.
> >
> >
> > Guozhang
> >
> > On Tue, Jul 30, 2019 at 1:55 PM Boyang Chen 
> > wrote:
> >
> > > Thank you Guozhang for the reply. We will consider the interface change
> > > from 429 as a backup plan for 447.
> > >
> > > And bumping this thread for more discussion.
> > >
> > > On Mon, Jul 22, 2019 at 6:28 PM Guozhang Wang 
> > wrote:
> > >
> > > > On Sat, Jul 20, 2019 at 9:50 AM Boyang Chen <
> > reluctanthero...@gmail.com>
> > > > wrote:
> > > >
> > > > > Thank you Guozhang for the suggestion! I would normally prefer
> > naming a
> > > > > flag corresponding to its functionality. Seems to me
> > `isolation_level`
> > > > > makes us another hop on information track.
> > > > >
> > > > > Fair enough, let's use a separate flag name then :)
> > > >
> > > >
> > > > > As for the generation.id exposure, I'm fine leveraging the new API
> > > from
> > > > > 429, but however is that design finalized yet, and whether the API
> > will
> > > > be
> > > > > added on the generic Consumer interface?
> > > > >
> > > > > The current PartitionAssignor is inside `internals` package and in
> > > > KIP-429
> > > > we are going to create a new interface out of `internals` to really
> > make
> > > it
> > > > public APIs, and as part of that we are refactoring some of its
> method
> > > > signatures. I just feel some of the newly introduced classes can be
> > > reused
> > > > in your KIP as well, i.e. just for code succinctness, but no
> semantical
> > > > indications.
> > > >
> > > >
> > > > > Boyang
> > > > >
> > > > > On Fri, Jul 19, 2019 at 3:57 PM Guozhang Wang 
> > > > wrote:
> > > > >
> > > > > > Boyang, thanks for the updated proposal!
> > > > > >
> > > > > > 3.a. As Jason mentioned, with EOS enabled we still need to
> augment
> > > the
> > > > > > offset fetch request with a boolean to indicate "give me an
> > retriable
> > > > > error
> > > > > > code if there's pending offset, rather than sending me the
> > committed
> > > > > offset
> > > > > > immediately". Personally I still feel it is okay to piggy-back on
> > the
> > > > > > ISOLATION_LEVEL boolean, but I'm also fine with another
> > > > > `await_transaction`
> > > > > > boolean if you feel strongly about it.
> > > > > >
> > > > > > 10. About the exposure of generation id, there may be some
> > > refactoring
> > > > > work
> > > > > > coming from KIP-429 that can benefit KIP-447 as well since we are
> > > > > wrapping
> > > > > > the consumer subscription / assignment data in new classes. Note
> > that
> > > > > > current proposal does not `generationId` since with the
> cooperative
> > > > > sticky
> > > > > > assignor we think it is not necessary for correctness, but also
> if
> > we
> > > > > agree
> > > > > > it is okay to expose it we can potentially include it in
> > > > > > `ConsumerAssignmentData` as well.
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > > On Thu, Jul 18, 2019 at 3:55 PM Boyang Chen <
> > > > reluctanthero...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Thank you Jason for the ideas.
> > > > > > >
> > > > > > > On Mon, Jul 15, 2019 at 5:28 PM Jason Gustafson <
> > > ja...@confluent.io>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi Boyang,
> > > > > > > >
> > > > > > > > Thanks for the updates. A few comments below:
> > > > > > > >
> > > > > > > > 1. The KIP mentions 

Re: [VOTE] KIP-440: Extend Connect Converter to support headers

2019-08-05 Thread Ryanne Dolan
+1, non-binding

Ryanne

On Mon, Aug 5, 2019 at 3:38 PM Randall Hauch  wrote:

> If my math is right, we have 3 non-binding +1 votes and 2 binding +1 votes.
>
> This is a simple but really beneficial KIP for Connect. Can we get another
> review and vote by a committer? Thanks!
>
> Randall
>
> On Fri, May 31, 2019 at 3:37 PM sapie...@gmail.com 
> wrote:
>
> > Hey everyone, just bumping this thread again. We need one more vote from
> > the committers. Thanks! :)
> >
> > On 2019/05/19 14:31:15, Kamal Chandraprakash <
> > kamal.chandraprak...@gmail.com> wrote:
> > > +1 (non-binding). Thanks for the KIP!
> > >
> > > On Sun, May 19, 2019 at 6:36 PM Dongjin Lee 
> wrote:
> > >
> > > > +1 (non-binding).
> > > >
> > > > Binding: +2 (Randall, Gwen)
> > > > Non-binding: +2 (Andrew, Dongjin)
> > > >
> > > > We need one more +1 from the committers. Is there anyone else?
> > > >
> > > > Thanks,
> > > > Dongjin
> > > >
> > > > On Fri, May 10, 2019 at 12:16 AM Andrew Schofield <
> > > > andrew_schofi...@live.com>
> > > > wrote:
> > > >
> > > > > +1 (non-binding).
> > > > >
> > > > > Looks good.
> > > > >
> > > > > On 09/05/2019, 15:55, "Gwen Shapira"  wrote:
> > > > >
> > > > > +1 (binding)
> > > > > Thank you!
> > > > >
> > > > > On Mon, May 6, 2019, 11:25 PM Yaroslav Tkachenko <
> > sapie...@gmail.com
> > > > >
> > > > > wrote:
> > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > > I'd like to start a vote for KIP-440: Extend Connect
> Converter
> > to
> > > > > support
> > > > > > headers (
> > > > > >
> > > > > >
> > > > >
> > > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-440%253A%2BExtend%2BConnect%2BConverter%2Bto%2Bsupport%2Bheaders=02%7C01%7C%7C82864a9a5f3049e8ca1d08d6d48e5147%7C84df9e7fe9f640afb435%7C1%7C0%7C636930105039335723=FyQT5pfTwtT2NTMStgSl6zRjiaWFVJqG3%2B4vt5nRYmI%3D=0
> > > > > > )
> > > > > >
> > > > > > Discussion:
> > > > > >
> > > > > >
> > > > >
> > > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Flists.apache.org%2Fthread.html%2F1fc1e3d2cddd8311d3db7c98f0d09a1a137ca4b20d1f3c8ab203a855%40%253Cdev.kafka.apache.org%253E=02%7C01%7C%7C82864a9a5f3049e8ca1d08d6d48e5147%7C84df9e7fe9f640afb435%7C1%7C0%7C636930105039335723=B2a4Mx9ScWaO3HEEw0LoIRX0ajETAcwUmDxt5Ir5FIs%3D=0
> > > > > >
> > > > > > Thanks!
> > > > > >
> > > > >
> > > > > --
> > > > > *Dongjin Lee*
> > > > >
> > > > > *A hitchhiker in the mathematical world.*
> > > > > *github:  github.com/dongjinleekr
> > > > > linkedin:
> > > > kr.linkedin.com/in/dongjinleekr
> > > > > speakerdeck:
> > > > speakerdeck.com/dongjin
> > > > > *
> > > > >
> > > >
> > >
> >
>


[jira] [Created] (KAFKA-8755) Stand-by Task of an Optimized Source Table Does Not Write Anything to its State Store

2019-08-05 Thread Bruno Cadonna (JIRA)
Bruno Cadonna created KAFKA-8755:


 Summary: Stand-by Task of an Optimized Source Table Does Not Write 
Anything to its State Store
 Key: KAFKA-8755
 URL: https://issues.apache.org/jira/browse/KAFKA-8755
 Project: Kafka
  Issue Type: Bug
  Components: streams
Affects Versions: 2.4.0
Reporter: Bruno Cadonna
 Attachments: StandbyTaskTest.java

With the following topology:

{code:java}
builder.table(
INPUT_TOPIC, 
Consumed.with(Serdes.Integer(), Serdes.Integer()), 
Materialized.>as(stateName)
)
{code}

and with topology optimization turned on, Kafka Streams uses the input topic 
{{INPUT_TOPIC}} as the change log topic for state store {{stateName}}. A 
stand-by task for such a topology should read from {{INPUT_TOPIC}} and should 
write the records to its state store so that the streams client that runs the 
stand-by task can take over the execution of the topology in case of a failure 
with an up-to-date replica of the state.

Currently, the stand-by task described above reads from the input topic but 
does not write the records to its state store. Thus, after a failure the 
stand-by task cannot provide any up-to-date state store and the streams client 
needs to construct the state from scratch before it can take over the execution.

The described behaviour can be reproduced with the attached test.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [VOTE] KIP-429: Kafka Consumer Incremental Rebalance Protocol

2019-08-05 Thread Guozhang Wang
Hello folks,

I've also updated the wiki page by making the augmented
`ConsumerPartitionAssignor` out as a public API in `o.a.k.clients.consumer`
and deprecate the old `PartitionAssignor` as in
`o.a.k.clients.consumer.internals`.


Guozhang

On Fri, Jun 28, 2019 at 11:30 AM Sophie Blee-Goldman 
wrote:

> It is now! Also updated the KIP to reflect that we will be adding a new
> CooperativeStickyAssignor rather than making the existing StickyAssignor
> cooperative to prevent users who already use the StickyAssignor from
> blindly upgrading and hitting potential problems during a rolling bounce
>
> On Thu, Jun 27, 2019 at 8:15 PM Boyang Chen 
> wrote:
>
> > Thank you Sophie for the update. Is this also reflected on the KIP?
> >
> > On Thu, Jun 27, 2019 at 3:28 PM Sophie Blee-Goldman  >
> > wrote:
> >
> > > We would like to tack on some rebalance-related metrics as part of this
> > KIP
> > > as well. The details can be found in the sub-task JIRA:
> > > https://issues.apache.org/jira/browse/KAFKA-8609
> > >
> > > On Thu, May 30, 2019 at 5:09 PM Guozhang Wang 
> > wrote:
> > >
> > > > +1 (binding) from me as well.
> > > >
> > > > Thanks to everyone who have voted! I'm closing this vote thread with
> a
> > > > tally:
> > > >
> > > > binding +1: 3 (Guozhang, Harsha, Matthias)
> > > >
> > > > non-binding +1: 2 (Boyang, Liquan)
> > > >
> > > >
> > > > Guozhang
> > > >
> > > > On Wed, May 22, 2019 at 9:22 PM Matthias J. Sax <
> matth...@confluent.io
> > >
> > > > wrote:
> > > >
> > > > > +1 (binding)
> > > > >
> > > > >
> > > > > On 5/22/19 7:37 PM, Harsha wrote:
> > > > > > +1 (binding). Thanks for the KIP looking forward for this to be
> > > > avaiable
> > > > > in consumers.
> > > > > >
> > > > > > Thanks,
> > > > > > Harsha
> > > > > >
> > > > > > On Wed, May 22, 2019, at 12:24 AM, Liquan Pei wrote:
> > > > > >> +1 (non-binding)
> > > > > >>
> > > > > >> On Tue, May 21, 2019 at 11:34 PM Boyang Chen <
> bche...@outlook.com
> > >
> > > > > wrote:
> > > > > >>
> > > > > >>> Thank you Guozhang for all the hard work.
> > > > > >>>
> > > > > >>> +1 (non-binding)
> > > > > >>>
> > > > > >>> 
> > > > > >>> From: Guozhang Wang 
> > > > > >>> Sent: Wednesday, May 22, 2019 1:32 AM
> > > > > >>> To: dev
> > > > > >>> Subject: [VOTE] KIP-429: Kafka Consumer Incremental Rebalance
> > > > Protocol
> > > > > >>>
> > > > > >>> Hello folks,
> > > > > >>>
> > > > > >>> I'd like to start the voting for KIP-429 now, details can be
> > found
> > > > > here:
> > > > > >>>
> > > > > >>>
> > > > > >>>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-429%3A+Kafka+Consumer+Incremental+Rebalance+Protocol#KIP-429:KafkaConsumerIncrementalRebalanceProtocol-RebalanceCallbackErrorHandling
> > > > > >>>
> > > > > >>> And the on-going PRs available for review:
> > > > > >>>
> > > > > >>> Part I: https://github.com/apache/kafka/pull/6528
> > > > > >>> Part II: https://github.com/apache/kafka/pull/6778
> > > > > >>>
> > > > > >>>
> > > > > >>> Thanks
> > > > > >>> -- Guozhang
> > > > > >>>
> > > > > >>
> > > > > >>
> > > > > >> --
> > > > > >> Liquan Pei
> > > > > >> Software Engineer, Confluent Inc
> > > > > >>
> > > > >
> > > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>


-- 
-- Guozhang


Re: [VOTE] KIP-440: Extend Connect Converter to support headers

2019-08-05 Thread Randall Hauch
If my math is right, we have 3 non-binding +1 votes and 2 binding +1 votes.

This is a simple but really beneficial KIP for Connect. Can we get another
review and vote by a committer? Thanks!

Randall

On Fri, May 31, 2019 at 3:37 PM sapie...@gmail.com 
wrote:

> Hey everyone, just bumping this thread again. We need one more vote from
> the committers. Thanks! :)
>
> On 2019/05/19 14:31:15, Kamal Chandraprakash <
> kamal.chandraprak...@gmail.com> wrote:
> > +1 (non-binding). Thanks for the KIP!
> >
> > On Sun, May 19, 2019 at 6:36 PM Dongjin Lee  wrote:
> >
> > > +1 (non-binding).
> > >
> > > Binding: +2 (Randall, Gwen)
> > > Non-binding: +2 (Andrew, Dongjin)
> > >
> > > We need one more +1 from the committers. Is there anyone else?
> > >
> > > Thanks,
> > > Dongjin
> > >
> > > On Fri, May 10, 2019 at 12:16 AM Andrew Schofield <
> > > andrew_schofi...@live.com>
> > > wrote:
> > >
> > > > +1 (non-binding).
> > > >
> > > > Looks good.
> > > >
> > > > On 09/05/2019, 15:55, "Gwen Shapira"  wrote:
> > > >
> > > > +1 (binding)
> > > > Thank you!
> > > >
> > > > On Mon, May 6, 2019, 11:25 PM Yaroslav Tkachenko <
> sapie...@gmail.com
> > > >
> > > > wrote:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > I'd like to start a vote for KIP-440: Extend Connect Converter
> to
> > > > support
> > > > > headers (
> > > > >
> > > > >
> > > >
> > >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-440%253A%2BExtend%2BConnect%2BConverter%2Bto%2Bsupport%2Bheaders=02%7C01%7C%7C82864a9a5f3049e8ca1d08d6d48e5147%7C84df9e7fe9f640afb435%7C1%7C0%7C636930105039335723=FyQT5pfTwtT2NTMStgSl6zRjiaWFVJqG3%2B4vt5nRYmI%3D=0
> > > > > )
> > > > >
> > > > > Discussion:
> > > > >
> > > > >
> > > >
> > >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Flists.apache.org%2Fthread.html%2F1fc1e3d2cddd8311d3db7c98f0d09a1a137ca4b20d1f3c8ab203a855%40%253Cdev.kafka.apache.org%253E=02%7C01%7C%7C82864a9a5f3049e8ca1d08d6d48e5147%7C84df9e7fe9f640afb435%7C1%7C0%7C636930105039335723=B2a4Mx9ScWaO3HEEw0LoIRX0ajETAcwUmDxt5Ir5FIs%3D=0
> > > > >
> > > > > Thanks!
> > > > >
> > > >
> > > > --
> > > > *Dongjin Lee*
> > > >
> > > > *A hitchhiker in the mathematical world.*
> > > > *github:  github.com/dongjinleekr
> > > > linkedin:
> > > kr.linkedin.com/in/dongjinleekr
> > > > speakerdeck:
> > > speakerdeck.com/dongjin
> > > > *
> > > >
> > >
> >
>


Re: [VOTE] KIP-396: Add Commit/List Offsets Operations to AdminClient

2019-08-05 Thread Colin McCabe
I think it would be useful to have this in AdminClient.  Especially if we 
implement KIP-496: Administrative API to delete consumer offsets.  It would be 
odd to have a way to delete consumer offsets in AdminClient, but not to create 
them.  What do you think?

best,
Colin


On Sun, Aug 4, 2019, at 09:27, Mickael Maison wrote:
> Hi,
> 
> In an attempt to unblock this KIP, I've made some adjustments:
> I've renamed the commitConsumerGroupOffsets() methods to
> resetConsumerGroupOffsets() to reduce confusion. That should better
> highlight the differences with the regular commit() operation from the
> Consumer API. I've also added some details to the motivation section.
> 
> So we have +5 non binding votes and 0 binding votes
> 
> On Mon, Mar 25, 2019 at 1:10 PM Mickael Maison  
> wrote:
> >
> > Bumping this thread once again
> >
> > Ismael, have I answered your questions?
> > While this has received a few non-binding +1s, no committers have
> > voted yet. If you have concerns or questions, please let me know.
> >
> > Thanks
> >
> > On Mon, Feb 11, 2019 at 11:51 AM Mickael Maison
> >  wrote:
> > >
> > > Bumping this thread as it's been a couple of weeks.
> > >
> > > On Tue, Jan 22, 2019 at 2:26 PM Mickael Maison  
> > > wrote:
> > > >
> > > > Thanks Ismael for the feedback. I think your point has 2 parts:
> > > > - Having the reset functionality in the AdminClient:
> > > > The fact we have a command line tool illustrate that this operation is
> > > > relatively common. I seems valuable to be able to perform this
> > > > operation directly via a proper API in addition of the CLI tool.
> > > >
> > > > - Sending an OffsetCommit directly instead of relying on KafkaConsumer:
> > > > The KafkaConsumer requires a lot of stuff to commit offsets. Its group
> > > > cannot change so you need to start a new Consumer every time, that
> > > > creates new connections and overal sends more requests. Also there are
> > > > already  a bunch of AdminClient APIs that have logic very close to
> > > > what needs to be done to send a commit request, keeping the code small
> > > > and consistent.
> > > >
> > > > I've updated the KIP with these details and moved the 2nd part to
> > > > "Proposed changes" as it's more an implementation detail.
> > > >
> > > > I hope this answers your question
> > > >
> > > > On Mon, Jan 21, 2019 at 7:41 PM Ismael Juma  wrote:
> > > > >
> > > > > The KIP doesn't discuss the option of using KafkaConsumer directly as 
> > > > > far
> > > > > as I can tell. We have tried to avoid having the same functionality in
> > > > > multiple clients so it would be good to explain why this is necessary 
> > > > > here
> > > > > (not saying it isn't).
> > > > >
> > > > > Ismael
> > > > >
> > > > > On Mon, Jan 21, 2019, 10:29 AM Mickael Maison 
> > > > >  > > > > wrote:
> > > > >
> > > > > > Thanks Ryanne for the feedback, all suggestions sounded good, I've
> > > > > > updated the KIP accordingly.
> > > > > >
> > > > > > On Mon, Jan 21, 2019 at 3:43 PM Ryanne Dolan 
> > > > > > wrote:
> > > > > > >
> > > > > > > +1 (non-binding)
> > > > > > >
> > > > > > > But I suggest:
> > > > > > >
> > > > > > > - drop "get" from getOffset, getTimestamp.
> > > > > > >
> > > > > > > - add to the motivation section why this is better than 
> > > > > > > constructing a
> > > > > > > KafkaConsumer and using seek(), commit() etc.
> > > > > > >
> > > > > > > - add some rejected alternatives.
> > > > > > >
> > > > > > > Ryanne
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Jan 21, 2019, 7:57 AM Dongjin Lee  > > > > > > wrote:
> > > > > > >
> > > > > > > > We have +4 non-binding for this vote. Is there any committer 
> > > > > > > > who is
> > > > > > > > interested in this issue?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Dongjin
> > > > > > > >
> > > > > > > > On Mon, Jan 21, 2019 at 10:33 PM Andrew Schofield <
> > > > > > > > andrew_schofi...@live.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > +1 (non-binding). Thanks for the KIP.
> > > > > > > > >
> > > > > > > > > On 21/01/2019, 12:45, "Eno Thereska" 
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > +1 (non binding). Thanks.
> > > > > > > > >
> > > > > > > > > On Mon, Jan 21, 2019 at 12:30 PM Mickael Maison <
> > > > > > > > > mickael.mai...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Bumping this thread. Considering this KIP is relatively 
> > > > > > > > > straigh
> > > > > > > > > > forward, can we get some votes or feedback if you think 
> > > > > > > > > it's
> > > > > > not?
> > > > > > > > > > Thanks
> > > > > > > > > >
> > > > > > > > > > On Tue, Jan 8, 2019 at 5:40 PM Edoardo Comar <
> > > > > > edoco...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > +1 (non-binding)
> > > > > > > > > > > Thanks Mickael!
> > > > > > > > > > >
> > > > > > > > > > > On Tue, 8 Jan 2019 at 17:39, Patrik Kleindl <
> > > > > > 

Re: [DISCUSS] KIP-503: deleted topics metric

2019-08-05 Thread Harsha Chintalapani
Thanks for the KIP.  Its useful metric to have.  LGTM.
-Harsha


On Mon, Aug 05, 2019 at 11:24 AM, David Arthur 
wrote:

> Hello all, I'd like to start a discussion for
> https://cwiki.apache.org/confluence/display/KAFKA/
> KIP-503%3A+Add+metric+for+number+of+topics+marked+for+deletion
>
> Thanks!
> David
>


[jira] [Created] (KAFKA-8754) Connect API: Expose the list of available transformations

2019-08-05 Thread JIRA
Stéphane Derosiaux created KAFKA-8754:
-

 Summary: Connect API: Expose the list of available transformations
 Key: KAFKA-8754
 URL: https://issues.apache.org/jira/browse/KAFKA-8754
 Project: Kafka
  Issue Type: Wish
  Components: KafkaConnect
Reporter: Stéphane Derosiaux


The API of Kafka Connect exposes the available connectors through: 
/connector-plugins/.

It would be useful to have another API method to expose the list of available 
transformations.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-05 Thread Colin McCabe
On Mon, Aug 5, 2019, at 10:02, Tom Bentley wrote:
> Hi Colin,
> 
> Thanks for the KIP.
> 
> Currently ZooKeeper provides a convenient notification mechanism for
> knowing that broker and topic configuration has changed. While KIP-500 does
> suggest that incremental metadata update is expected to come to clients
> eventually, that would seem to imply that for some number of releases there
> would be no equivalent mechanism for knowing about config changes. Is there
> any thinking at this point about how a similar notification might be
> provided in the future?

We could eventually have some inotify-like mechanism where clients could 
register interest in various types of events and got notified when they 
happened.  Reading the metadata log is conceptually simple.  The main 
complexity would be in setting up an API that made sense and that didn't unduly 
constrain future implementations.  We'd have to think carefully about what the 
real use-cases for this were, though.

best,
Colin

> 
> Thanks,
> 
> Tom
> 
> On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass 
> wrote:
> 
> > Hey Colin,
> >
> > I think this is a long-awaited KIP, thanks for driving it. I'm excited to
> > see this in Kafka once. I collected my questions (and I accept the "TBD"
> > answer as they might be a bit deep for this high level :) ).
> > 1.) Are there any specific reasons for the Controller just periodically
> > persisting its state on disk periodically instead of asynchronously with
> > every update? Wouldn't less frequent saves increase the chance for missing
> > a state change if the controller crashes between two saves?
> > 2.) Why can't we allow brokers to fetch metadata from the follower
> > controllers? I assume that followers would have up-to-date information
> > therefore brokers could fetch from there in theory.
> >
> > Thanks,
> > Viktor
> >
> > On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen 
> > wrote:
> >
> > > Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds
> > like
> > > a good idea.
> > >
> > > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma  wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Yes, there will be several KIPs that will discuss the items you
> > describe
> > > in
> > > > detail. Colin, it may be helpful to make this clear in the KIP 500
> > > > description.
> > > >
> > > > Ismael
> > > >
> > > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen  > >
> > > > wrote:
> > > >
> > > > > Thanks Colin for initiating this important effort!
> > > > >
> > > > > One question I have is whether we have a session discussing the
> > > > controller
> > > > > failover in the new architecture? I know we are using Raft protocol
> > to
> > > > > failover, yet it's still valuable to discuss the steps new cluster is
> > > > going
> > > > > to take to reach the stable stage again, so that we could easily
> > > measure
> > > > > the availability of the metadata servers.
> > > > >
> > > > > Another suggestion I have is to write a step-by-step design doc like
> > > what
> > > > > we did in KIP-98
> > > > > <
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > > >,
> > > > > including the new request protocols and how they are interacting in
> > the
> > > > new
> > > > > cluster. For a complicated change like this, an implementation design
> > > doc
> > > > > help a lot in the review process, otherwise most discussions we have
> > > will
> > > > > focus on high level and lose important details as we discover them in
> > > the
> > > > > post-agreement phase.
> > > > >
> > > > > Boyang
> > > > >
> > > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe 
> > > wrote:
> > > > >
> > > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > > > Thanks Colin for the detail KIP. I have a few comments and
> > > questions.
> > > > > > >
> > > > > > > In the KIP's Motivation and Overview you mentioned the
> > LeaderAndIsr
> > > > and
> > > > > > > UpdateMetadata RPC. For example, "updates which the controller
> > > > pushes,
> > > > > > such
> > > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking
> > that
> > > > we
> > > > > > will
> > > > > > > use MetadataFetch as a replacement to just UpdateMetadata only
> > and
> > > > add
> > > > > > > topic configuration in this state?
> > > > > > >
> > > > > >
> > > > > > Hi Jose,
> > > > > >
> > > > > > Thanks for taking a look.
> > > > > >
> > > > > > The goal is for MetadataFetchRequest to replace both
> > > > LeaderAndIsrRequest
> > > > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> > > along
> > > > > > with the other metadata.
> > > > > >
> > > > > > > In the section "Broker Metadata Management", you mention "Just
> > like
> > > > > with
> > > > > > a
> > > > > > > fetch request, the broker will track the offset of the last
> > updates
> > > > it
> > > > > > > fetched". To keep the log consistent Raft requires that the
> > > 

Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-05 Thread Colin McCabe
On Mon, Aug 5, 2019, at 07:49, Viktor Somogyi-Vass wrote:
> Hey Colin,
> 
> I think this is a long-awaited KIP, thanks for driving it. I'm excited to
> see this in Kafka once. I collected my questions (and I accept the "TBD"
> answer as they might be a bit deep for this high level :) ).
> 1.) Are there any specific reasons for the Controller just periodically
> persisting its state on disk periodically instead of asynchronously with
> every update? Wouldn't less frequent saves increase the chance for missing
> a state change if the controller crashes between two saves?

Hi Viktor,

To be clear, the controller will persist its state on disk with every update.  
It's just that the state will be persisted in the form of a write-ahead log 
entry.  The periodic snapshots are to shorten loading times, not a mechanism 
for durability.

> 2.) Why can't we allow brokers to fetch metadata from the follower
> controllers? I assume that followers would have up-to-date information
> therefore brokers could fetch from there in theory.

In Raft, the leader always has the most up-to-date information.  A follower may 
not have up-to-date information.  For example, it may not have been part of the 
majority that wrote the latest update.

We also want the controller to know when brokers stop fetching metadata, so 
that they can be transitioned to a fenced state.  This requires fetching from 
the controller.

best,
Colin

> 
> Thanks,
> Viktor
> 
> On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen 
> wrote:
> 
> > Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds like
> > a good idea.
> >
> > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma  wrote:
> >
> > > Hi Boyang,
> > >
> > > Yes, there will be several KIPs that will discuss the items you describe
> > in
> > > detail. Colin, it may be helpful to make this clear in the KIP 500
> > > description.
> > >
> > > Ismael
> > >
> > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen 
> > > wrote:
> > >
> > > > Thanks Colin for initiating this important effort!
> > > >
> > > > One question I have is whether we have a session discussing the
> > > controller
> > > > failover in the new architecture? I know we are using Raft protocol to
> > > > failover, yet it's still valuable to discuss the steps new cluster is
> > > going
> > > > to take to reach the stable stage again, so that we could easily
> > measure
> > > > the availability of the metadata servers.
> > > >
> > > > Another suggestion I have is to write a step-by-step design doc like
> > what
> > > > we did in KIP-98
> > > > <
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >,
> > > > including the new request protocols and how they are interacting in the
> > > new
> > > > cluster. For a complicated change like this, an implementation design
> > doc
> > > > help a lot in the review process, otherwise most discussions we have
> > will
> > > > focus on high level and lose important details as we discover them in
> > the
> > > > post-agreement phase.
> > > >
> > > > Boyang
> > > >
> > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe 
> > wrote:
> > > >
> > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > > Thanks Colin for the detail KIP. I have a few comments and
> > questions.
> > > > > >
> > > > > > In the KIP's Motivation and Overview you mentioned the LeaderAndIsr
> > > and
> > > > > > UpdateMetadata RPC. For example, "updates which the controller
> > > pushes,
> > > > > such
> > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking that
> > > we
> > > > > will
> > > > > > use MetadataFetch as a replacement to just UpdateMetadata only and
> > > add
> > > > > > topic configuration in this state?
> > > > > >
> > > > >
> > > > > Hi Jose,
> > > > >
> > > > > Thanks for taking a look.
> > > > >
> > > > > The goal is for MetadataFetchRequest to replace both
> > > LeaderAndIsrRequest
> > > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> > along
> > > > > with the other metadata.
> > > > >
> > > > > > In the section "Broker Metadata Management", you mention "Just like
> > > > with
> > > > > a
> > > > > > fetch request, the broker will track the offset of the last updates
> > > it
> > > > > > fetched". To keep the log consistent Raft requires that the
> > followers
> > > > > keep
> > > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> > return
> > > > log
> > > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > > which
> > > > > > will replicate/fetch log entries after the highwatermark for
> > > consensus?
> > > > >
> > > > > Good question.  Clearly, we shouldn't expose metadata updates to the
> > > > > brokers until they've been stored on a majority of the Raft nodes.
> > The
> > 

[DISCUSS] KIP-503: deleted topics metric

2019-08-05 Thread David Arthur
Hello all, I'd like to start a discussion for
https://cwiki.apache.org/confluence/display/KAFKA/KIP-503%3A+Add+metric+for+number+of+topics+marked+for+deletion

Thanks!
David


Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-05 Thread Tom Bentley
Hi Colin,

Thanks for the KIP.

Currently ZooKeeper provides a convenient notification mechanism for
knowing that broker and topic configuration has changed. While KIP-500 does
suggest that incremental metadata update is expected to come to clients
eventually, that would seem to imply that for some number of releases there
would be no equivalent mechanism for knowing about config changes. Is there
any thinking at this point about how a similar notification might be
provided in the future?

Thanks,

Tom

On Mon, Aug 5, 2019 at 3:49 PM Viktor Somogyi-Vass 
wrote:

> Hey Colin,
>
> I think this is a long-awaited KIP, thanks for driving it. I'm excited to
> see this in Kafka once. I collected my questions (and I accept the "TBD"
> answer as they might be a bit deep for this high level :) ).
> 1.) Are there any specific reasons for the Controller just periodically
> persisting its state on disk periodically instead of asynchronously with
> every update? Wouldn't less frequent saves increase the chance for missing
> a state change if the controller crashes between two saves?
> 2.) Why can't we allow brokers to fetch metadata from the follower
> controllers? I assume that followers would have up-to-date information
> therefore brokers could fetch from there in theory.
>
> Thanks,
> Viktor
>
> On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen 
> wrote:
>
> > Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds
> like
> > a good idea.
> >
> > On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma  wrote:
> >
> > > Hi Boyang,
> > >
> > > Yes, there will be several KIPs that will discuss the items you
> describe
> > in
> > > detail. Colin, it may be helpful to make this clear in the KIP 500
> > > description.
> > >
> > > Ismael
> > >
> > > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen  >
> > > wrote:
> > >
> > > > Thanks Colin for initiating this important effort!
> > > >
> > > > One question I have is whether we have a session discussing the
> > > controller
> > > > failover in the new architecture? I know we are using Raft protocol
> to
> > > > failover, yet it's still valuable to discuss the steps new cluster is
> > > going
> > > > to take to reach the stable stage again, so that we could easily
> > measure
> > > > the availability of the metadata servers.
> > > >
> > > > Another suggestion I have is to write a step-by-step design doc like
> > what
> > > > we did in KIP-98
> > > > <
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > > >,
> > > > including the new request protocols and how they are interacting in
> the
> > > new
> > > > cluster. For a complicated change like this, an implementation design
> > doc
> > > > help a lot in the review process, otherwise most discussions we have
> > will
> > > > focus on high level and lose important details as we discover them in
> > the
> > > > post-agreement phase.
> > > >
> > > > Boyang
> > > >
> > > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe 
> > wrote:
> > > >
> > > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > > Thanks Colin for the detail KIP. I have a few comments and
> > questions.
> > > > > >
> > > > > > In the KIP's Motivation and Overview you mentioned the
> LeaderAndIsr
> > > and
> > > > > > UpdateMetadata RPC. For example, "updates which the controller
> > > pushes,
> > > > > such
> > > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking
> that
> > > we
> > > > > will
> > > > > > use MetadataFetch as a replacement to just UpdateMetadata only
> and
> > > add
> > > > > > topic configuration in this state?
> > > > > >
> > > > >
> > > > > Hi Jose,
> > > > >
> > > > > Thanks for taking a look.
> > > > >
> > > > > The goal is for MetadataFetchRequest to replace both
> > > LeaderAndIsrRequest
> > > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> > along
> > > > > with the other metadata.
> > > > >
> > > > > > In the section "Broker Metadata Management", you mention "Just
> like
> > > > with
> > > > > a
> > > > > > fetch request, the broker will track the offset of the last
> updates
> > > it
> > > > > > fetched". To keep the log consistent Raft requires that the
> > followers
> > > > > keep
> > > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> > return
> > > > log
> > > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > > which
> > > > > > will replicate/fetch log entries after the highwatermark for
> > > consensus?
> > > > >
> > > > > Good question.  Clearly, we shouldn't expose metadata updates to
> the
> > > > > brokers until they've been stored on a majority of the Raft nodes.
> > The
> > > > > most obvious way to do that, like you mentioned, is to have the
> > brokers
> > > > > only fetch up to the HWM, but 

Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics

2019-08-05 Thread Boyang Chen
Thank you Guozhang for the reply! I'm curious whether KIP-429 has reflected
the latest change on ConsumerGroupMetadata? Also regarding question one,
the group metadata needs to be accessed via callback, does that mean we
need a separate producer API such like
"producer.refreshMetadata(groupMetadata)" to be able to access it instead
of passing in the consumer instance?

Boyang

On Fri, Aug 2, 2019 at 4:36 PM Guozhang Wang  wrote:

> Thanks Boyang,
>
> I've made another pass on KIP-447 as well as
> https://github.com/apache/kafka/pull/7078, and have some minor comments
> about the proposed API:
>
> 1. it seems instead of needing the whole KafkaConsumer object, you'd only
> need the "ConsumerGroupMetadata", in that case can we just pass in that
> object into the initTxns call?
>
> 2. the current trunk already has a public class named
> (ConsumerGroupMetadata)
> under o.a.k.clients.consumer created by KIP-429. If we want to just use
> that then maybe it makes less sense to declare a base GroupMetadata as we
> are already leaking such information on the assignor anyways.
>
>
> Guozhang
>
> On Tue, Jul 30, 2019 at 1:55 PM Boyang Chen 
> wrote:
>
> > Thank you Guozhang for the reply. We will consider the interface change
> > from 429 as a backup plan for 447.
> >
> > And bumping this thread for more discussion.
> >
> > On Mon, Jul 22, 2019 at 6:28 PM Guozhang Wang 
> wrote:
> >
> > > On Sat, Jul 20, 2019 at 9:50 AM Boyang Chen <
> reluctanthero...@gmail.com>
> > > wrote:
> > >
> > > > Thank you Guozhang for the suggestion! I would normally prefer
> naming a
> > > > flag corresponding to its functionality. Seems to me
> `isolation_level`
> > > > makes us another hop on information track.
> > > >
> > > > Fair enough, let's use a separate flag name then :)
> > >
> > >
> > > > As for the generation.id exposure, I'm fine leveraging the new API
> > from
> > > > 429, but however is that design finalized yet, and whether the API
> will
> > > be
> > > > added on the generic Consumer interface?
> > > >
> > > > The current PartitionAssignor is inside `internals` package and in
> > > KIP-429
> > > we are going to create a new interface out of `internals` to really
> make
> > it
> > > public APIs, and as part of that we are refactoring some of its method
> > > signatures. I just feel some of the newly introduced classes can be
> > reused
> > > in your KIP as well, i.e. just for code succinctness, but no semantical
> > > indications.
> > >
> > >
> > > > Boyang
> > > >
> > > > On Fri, Jul 19, 2019 at 3:57 PM Guozhang Wang 
> > > wrote:
> > > >
> > > > > Boyang, thanks for the updated proposal!
> > > > >
> > > > > 3.a. As Jason mentioned, with EOS enabled we still need to augment
> > the
> > > > > offset fetch request with a boolean to indicate "give me an
> retriable
> > > > error
> > > > > code if there's pending offset, rather than sending me the
> committed
> > > > offset
> > > > > immediately". Personally I still feel it is okay to piggy-back on
> the
> > > > > ISOLATION_LEVEL boolean, but I'm also fine with another
> > > > `await_transaction`
> > > > > boolean if you feel strongly about it.
> > > > >
> > > > > 10. About the exposure of generation id, there may be some
> > refactoring
> > > > work
> > > > > coming from KIP-429 that can benefit KIP-447 as well since we are
> > > > wrapping
> > > > > the consumer subscription / assignment data in new classes. Note
> that
> > > > > current proposal does not `generationId` since with the cooperative
> > > > sticky
> > > > > assignor we think it is not necessary for correctness, but also if
> we
> > > > agree
> > > > > it is okay to expose it we can potentially include it in
> > > > > `ConsumerAssignmentData` as well.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Thu, Jul 18, 2019 at 3:55 PM Boyang Chen <
> > > reluctanthero...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Thank you Jason for the ideas.
> > > > > >
> > > > > > On Mon, Jul 15, 2019 at 5:28 PM Jason Gustafson <
> > ja...@confluent.io>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Boyang,
> > > > > > >
> > > > > > > Thanks for the updates. A few comments below:
> > > > > > >
> > > > > > > 1. The KIP mentions that `transaction.timeout.ms` should be
> > > reduced
> > > > to
> > > > > > > 10s.
> > > > > > > I think this makes sense for Kafka Streams which is tied to the
> > > > > consumer
> > > > > > > group semantics and uses a default 10s session timeout.
> However,
> > it
> > > > > > seems a
> > > > > > > bit dangerous to make this change for the producer generally.
> > Could
> > > > we
> > > > > > just
> > > > > > > change it for streams?
> > > > > > >
> > > > > > > That sounds good to me.
> > > > > >
> > > > > > > 2. The new `initTransactions` API takes a `Consumer` instance.
> I
> > > > think
> > > > > > the
> > > > > > > idea is to basically put in a backdoor to give the producer
> > access
> > > to
> > > > > the
> > > > > > > group generationId. It's not clear to me how this 

[jira] [Created] (KAFKA-8753) Add JMX for number of topics marked for deletion

2019-08-05 Thread David Arthur (JIRA)
David Arthur created KAFKA-8753:
---

 Summary: Add JMX for number of topics marked for deletion
 Key: KAFKA-8753
 URL: https://issues.apache.org/jira/browse/KAFKA-8753
 Project: Kafka
  Issue Type: Improvement
  Components: metrics
Reporter: David Arthur
Assignee: David Arthur






--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [DISCUSS] KIP-500: Replace ZooKeeper with a Self-Managed Metadata Quorum

2019-08-05 Thread Viktor Somogyi-Vass
Hey Colin,

I think this is a long-awaited KIP, thanks for driving it. I'm excited to
see this in Kafka once. I collected my questions (and I accept the "TBD"
answer as they might be a bit deep for this high level :) ).
1.) Are there any specific reasons for the Controller just periodically
persisting its state on disk periodically instead of asynchronously with
every update? Wouldn't less frequent saves increase the chance for missing
a state change if the controller crashes between two saves?
2.) Why can't we allow brokers to fetch metadata from the follower
controllers? I assume that followers would have up-to-date information
therefore brokers could fetch from there in theory.

Thanks,
Viktor

On Sun, Aug 4, 2019 at 6:58 AM Boyang Chen 
wrote:

> Thanks for explaining Ismael! Breaking down into follow-up KIPs sounds like
> a good idea.
>
> On Sat, Aug 3, 2019 at 10:14 AM Ismael Juma  wrote:
>
> > Hi Boyang,
> >
> > Yes, there will be several KIPs that will discuss the items you describe
> in
> > detail. Colin, it may be helpful to make this clear in the KIP 500
> > description.
> >
> > Ismael
> >
> > On Sat, Aug 3, 2019 at 9:32 AM Boyang Chen 
> > wrote:
> >
> > > Thanks Colin for initiating this important effort!
> > >
> > > One question I have is whether we have a session discussing the
> > controller
> > > failover in the new architecture? I know we are using Raft protocol to
> > > failover, yet it's still valuable to discuss the steps new cluster is
> > going
> > > to take to reach the stable stage again, so that we could easily
> measure
> > > the availability of the metadata servers.
> > >
> > > Another suggestion I have is to write a step-by-step design doc like
> what
> > > we did in KIP-98
> > > <
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >,
> > > including the new request protocols and how they are interacting in the
> > new
> > > cluster. For a complicated change like this, an implementation design
> doc
> > > help a lot in the review process, otherwise most discussions we have
> will
> > > focus on high level and lose important details as we discover them in
> the
> > > post-agreement phase.
> > >
> > > Boyang
> > >
> > > On Fri, Aug 2, 2019 at 5:17 PM Colin McCabe 
> wrote:
> > >
> > > > On Fri, Aug 2, 2019, at 16:33, Jose Armando Garcia Sancio wrote:
> > > > > Thanks Colin for the detail KIP. I have a few comments and
> questions.
> > > > >
> > > > > In the KIP's Motivation and Overview you mentioned the LeaderAndIsr
> > and
> > > > > UpdateMetadata RPC. For example, "updates which the controller
> > pushes,
> > > > such
> > > > > as LeaderAndIsr and UpdateMetadata messages". Is your thinking that
> > we
> > > > will
> > > > > use MetadataFetch as a replacement to just UpdateMetadata only and
> > add
> > > > > topic configuration in this state?
> > > > >
> > > >
> > > > Hi Jose,
> > > >
> > > > Thanks for taking a look.
> > > >
> > > > The goal is for MetadataFetchRequest to replace both
> > LeaderAndIsrRequest
> > > > and UpdateMetadataRequest.  Topic configurations would be fetched
> along
> > > > with the other metadata.
> > > >
> > > > > In the section "Broker Metadata Management", you mention "Just like
> > > with
> > > > a
> > > > > fetch request, the broker will track the offset of the last updates
> > it
> > > > > fetched". To keep the log consistent Raft requires that the
> followers
> > > > keep
> > > > > all of the log entries (term/epoch and offset) that are after the
> > > > > highwatermark. Any log entry before the highwatermark can be
> > > > > compacted/snapshot. Do we expect the MetadataFetch API to only
> return
> > > log
> > > > > entries up to the highwatermark?  Unlike the Raft replication API
> > which
> > > > > will replicate/fetch log entries after the highwatermark for
> > consensus?
> > > >
> > > > Good question.  Clearly, we shouldn't expose metadata updates to the
> > > > brokers until they've been stored on a majority of the Raft nodes.
> The
> > > > most obvious way to do that, like you mentioned, is to have the
> brokers
> > > > only fetch up to the HWM, but not beyond.  There might be a more
> clever
> > > way
> > > > to do it by fetching the data, but not having the brokers act on it
> > until
> > > > the HWM advances.  I'm not sure if that's worth it or not.  We'll
> > discuss
> > > > this more in a separate KIP that just discusses just Raft.
> > > >
> > > > >
> > > > > In section "Broker Metadata Management", you mention "the
> controller
> > > will
> > > > > send a full metadata image rather than a series of deltas". This
> KIP
> > > > > doesn't go into the set of operations that need to be supported on
> > top
> > > of
> > > > > Raft but it would be interested if this "full metadata image" could
> > be
> > > > > express also as deltas. For example, assuming we are replicating a
> > map
> > > > this
> > > > > "full metadata image" could be a sequence of "put" 

Re: [VOTE] KIP-419: Safely notify Kafka Connect SourceTask is stopped

2019-08-05 Thread Andrew Schofield
Hi,
I'd like to have a final try at getting some voting on this KIP. I'd like to 
get it into Kafka 2.4 so get your votes in please.

Currently only +2 non-binding votes.

KIP: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-419%3A+Safely+notify+Kafka+Connect+SourceTask+is+stopped
PR: https://github.com/apache/kafka/pull/6551

The idea is that source connectors are multi-threaded and the signal to stop a 
running connector can be delivered while messages are being requested from the 
source system. It's not rocket science to handle clean up correctly in a 
connector, but you might need to create a thread to wait for activity to 
quieten down so you can do it safely. This KIP just gives a way to be called by 
the KC framework when the connector has properly quiesced. Makes the connector 
code a bit simpler, and that's particularly helpful for someone just knocking 
up a simple connector without worrying about thread management.

Thanks,
Andrew Schofield

On 25/04/2019, 16:11, "Andrew Schofield"  wrote:

I'd like to encourage some more votes on KIP-419. It's a pretty small KIP 
to make it easier to handle resource clean up in Kafka Connect SourceTasks.

Currently only +2 non-binding.

KIP: 
https://cwiki.apache.org/confluence/display/KAFKA/KIP-419%3A+Safely+notify+Kafka+Connect+SourceTask+is+stopped
PR: https://github.com/apache/kafka/pull/6551

Thanks,
Andrew Schofield
IBM Event Streams

On 15/04/2019, 15:59, "Edoardo Comar"  wrote:

Thanks Andrew.

+1 (non-binding)

--

Edoardo Comar

IBM Event Streams
IBM UK Ltd, Hursley Park, SO21 2JN




From:   Mickael Maison 
To: dev 
Date:   10/04/2019 10:14
Subject:Re: [VOTE] KIP-419: Safely notify Kafka Connect 
SourceTask 
is stopped



+1 (non-binding)
Thanks for the KIP!

On Mon, Apr 8, 2019 at 8:07 PM Andrew Schofield
 wrote:
>
> Hi,
> I’d like to begin the voting thread for KIP-419. This is a minor KIP 
to 
add a new stopped() method to the SourceTask interface in Kafka 
Connect. 
Its purpose is to give the task a safe opportunity to clean up its 
resources, in the knowledge that this is the final call to the task.
>
> KIP: 

https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Furldefense.proofpoint.com%2Fv2%2Furl%3Fu%3Dhttps-3A__cwiki.apache.org_confluence_display_KAFKA_KIP-2D419-253A-2BSafely-2Bnotify-2BKafka-2BConnect-2BSourceTask-2Bis-2Bstopped%26d%3DDwIFaQ%26c%3Djf_iaSHvJObTbx-siA1ZOg%26r%3DEzRhmSah4IHsUZVekRUIINhltZK7U0OaeRo7hgW4_tQ%26m%3DvBvXztcRTgKwMpQ54ziN_GoOo0_fHSvTEMoXwQABvfs%26s%3DParyN6mWVuOGJR7kA84NOshRJA2LAK6htiD2gqf-h_M%26edata=02%7C01%7C%7Cffb96440b207419a7fcd08d6c1b2ed17%7C84df9e7fe9f640afb435%7C1%7C0%7C636909371551269648sdata=WJ4ZgMEIUTl83QXBIm%2Fn3ekWWabpZTIWsPbQOQGR6J8%3Dreserved=0=

> PR: 

https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Furldefense.proofpoint.com%2Fv2%2Furl%3Fu%3Dhttps-3A__github.com_apache_kafka_pull_6551%26d%3DDwIFaQ%26c%3Djf_iaSHvJObTbx-siA1ZOg%26r%3DEzRhmSah4IHsUZVekRUIINhltZK7U0OaeRo7hgW4_tQ%26m%3DvBvXztcRTgKwMpQ54ziN_GoOo0_fHSvTEMoXwQABvfs%26s%3DR_udYap1tpd83ISv1Rh0TY6ttH6RuEIwQ0KwOFMB3zU%26edata=02%7C01%7C%7Cffb96440b207419a7fcd08d6c1b2ed17%7C84df9e7fe9f640afb435%7C1%7C0%7C636909371551279653sdata=4LrcZVcLG9acQm7rjZz8%2F9MO2UeKK08242TW1SSJdlE%3Dreserved=0=

> JIRA: 

https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Furldefense.proofpoint.com%2Fv2%2Furl%3Fu%3Dhttps-3A__issues.apache.org_jira_browse_KAFKA-2D7841%26d%3DDwIFaQ%26c%3Djf_iaSHvJObTbx-siA1ZOg%26r%3DEzRhmSah4IHsUZVekRUIINhltZK7U0OaeRo7hgW4_tQ%26m%3DvBvXztcRTgKwMpQ54ziN_GoOo0_fHSvTEMoXwQABvfs%26s%3D5WqDQPU2J8yAxRXsjOgydtzJSE8yQCoB7qX0TtQyHA0%26edata=02%7C01%7C%7Cffb96440b207419a7fcd08d6c1b2ed17%7C84df9e7fe9f640afb435%7C1%7C0%7C636909371551279653sdata=JxRlNBP9FmuCmSVHIj6T30eT3uMPijbHi%2B%2F1QsUfA5U%3Dreserved=0=

>
> Thanks,
> Andrew Schofield
> IBM




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







[jira] [Created] (KAFKA-8752) Ensure plugin classes are instantiable when discovering plugins

2019-08-05 Thread Alexandre Dupriez (JIRA)
Alexandre Dupriez created KAFKA-8752:


 Summary: Ensure plugin classes are instantiable when discovering 
plugins
 Key: KAFKA-8752
 URL: https://issues.apache.org/jira/browse/KAFKA-8752
 Project: Kafka
  Issue Type: Bug
  Components: core
Reporter: Alexandre Dupriez
Assignee: Alexandre Dupriez


While running integration tests from the IntelliJ IDE, it appears plugins fail 
to load in {{DelegatingClassLoader.scanUrlsAndAddPlugins}}. The reason was, in 
this case, that the class 
{{org.apache.kafka.connect.connector.ConnectorReconfigurationTest$TestConnector}}
 could not be instantiated - which it does not intend to be.

The problem does not when running integration tests with Gradle as the runtime 
closure is different from IntelliJ - which includes test sources from modules 
depended on on the classpath.

While debugging this minor inconvenience, I could see that 
{{DelegatingClassLoader}} performs a sanity check on the plugin class to 
instantiate - as of now, it verifies the class is concrete. A quick fix for the 
problem highlighted above could to add an extra condition on the Java modifiers 
of the class to ensure it will be instantiable.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (KAFKA-8751) Trogdor - Add endpoint that deletes all tasks

2019-08-05 Thread Stanislav Kozlovski (JIRA)
Stanislav Kozlovski created KAFKA-8751:
--

 Summary: Trogdor - Add endpoint that deletes all tasks
 Key: KAFKA-8751
 URL: https://issues.apache.org/jira/browse/KAFKA-8751
 Project: Kafka
  Issue Type: Improvement
Reporter: Stanislav Kozlovski


It is very useful to have a single endpoint which clears all of Trogdor's 
state. During testing, we sometimes want to start from a clean slate and ensure 
all tasks are finished.

The easiest way to do this is to delete all of them, ensuring all have stop. 
Currently, we need to resort to calling the API N times, where N is the number 
of (running) tasks present.

 

It would be way cooler if we had a single endpoint to do all of this for us, 
something like:

`DELETE /coordinator/tasks/`



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (KAFKA-8750) Utils#murmur2() method switch case block don't have break

2019-08-05 Thread yangkun (JIRA)
yangkun created KAFKA-8750:
--

 Summary: Utils#murmur2() method switch case block don't have break
 Key: KAFKA-8750
 URL: https://issues.apache.org/jira/browse/KAFKA-8750
 Project: Kafka
  Issue Type: Bug
  Components: clients
Reporter: yangkun


Utils#murmur2() is as following, switch case block don't have break.

 
{code:java}
public static int murmur2(final byte[] data) {
...

switch (length % 4) {
case 3:
h ^= (data[(length & ~3) + 2] & 0xff) << 16;
// no break
case 2:
h ^= (data[(length & ~3) + 1] & 0xff) << 8;
// no break
case 1:
h ^= data[length & ~3] & 0xff;
h *= m;
}

   ...

return h;
}
{code}
 

 



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)