[KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread Bart Vercammen
Hi,

I recently moved some KafkaStreams applications from v0.10.2.1 to v1.1.1
and now I notice a weird behaviour in the partition assignment.
When starting 4 instances of my Kafka Streams application (on v1.1.1) I see
that 17 of the 20 partitions (of a source topic) are assigned to 1 instance
of the application while the other 3 instances only get 1 partition
assigned. (previously (on v0.10.2.1) the all got 5 partitions.)

Is this expected behaviour, as I read that quite some improvements were
done in the partition assignment strategy for Kafka Streams applications?
If yes, how can I make it so that the partitions are equally devided again
across all running applications?   It's a bit weird in my opinion as this
makes scaling the application very hard.

Also, when initially starting with 1 instance of the application, and
gradually scaling up, the new instances only get 1 partition assigned ...

All my Streams applications use default configuration (more or less),
running 1 stream-thread.

Any suggestions / enlightenments on this?
Greets,
Bart


Re: Discussion on requirements for Data Encryption functionality in Kafka (KIP-317)

2018-10-08 Thread Sönke Liebau
Hi Mike,

that sounds good! I've not yet received any other feedback, but worst case
scenario is that just the two of us discuss this over a cup of coffee :)

I'll talk to the Summit organizers again about some sort of venue and get
back to you once I know more. Maybe we can get a few more people to join
the discussion once some more details are known.

Other than that, I am also very happy to gather feedback on the mailing
list from people who won't be able to make it to the Summit. So anybody who
can come up with some thoughts or requirements around encryption
functionality for Kafka, please don't hesitate to chime in!

Best regards,
Sönke

On Wed, Oct 3, 2018 at 3:03 AM mikegray...@gmail.com 
wrote:

> Hi Sönke,
>
> I would be very interested in participating in this conversation.  Very
> interested in how TDE might work in Kafka!  I’m coming with several
> colleagues and will see if they’re interested in participating as well.
>
> Thanks,
> Mike Grayson
>
> On 2018/10/02 11:19:36, Sönke Liebau 
> wrote:
> > Hi all,
> >
> > I have created KIP-317 [1] a while ago, which outlines an implementation
> > proposal to add transparent data encryption functionality to Kafka. The
> KIP
> > in its current form is somewhat rigid in its implementation, I will
> rework
> > this to become extensible over the next few days to allow for additional
> > implementations.
> >
> > I have discussed the current method of providing keys with a colleague
> and
> > while we agree that this is a valid use case for some people, there are
> > certainly a lot of other valid use cases out there as well.
> > To ensure that the initial implementation provides the necessary
> > flexibility I'd like some feedback from the community on what
> requirements
> > they would have around data encryption and key management.
> >
> > The following questions should serve as a starting point for the
> > discussion, please feel free to address anything that comes to mind
> which I
> > have not mentioned here:
> >
> > - Should encryption be configurable rather on the client or on the broker
> > and be pushed down to the client?
> > - Where should keys be stored?
> > - How much flexibility around keys is necessary - is there for example a
> > use case that would decide on a per message basis which key to use?
> > (imagine a topic containing top secret, secret and public data with three
> > different keys)
> > - Do we need functionality to prohibit publishing unencrypted messages to
> > topics based on that topics setup?
> >
> > Of course the mailing list is the first place that discussions like these
> > should take place, but sometimes I find a face to face discussion can be
> > quite useful as well, especially when discussing non-trivial topics (like
> > encryption). I have reached out to the organizers of the upcoming Kafka
> > Summit in SF and there might be a chance for us to get a room with a
> > whiteboard at some point (probably during lunch, when the room is
> otherwise
> > unused). Would people be interested in meeting up for 20 minutes to
> discuss
> > this in person? I'd be happy to provide a summary on the mailing list
> > afterwards of course.
> >
> > Look forward to hearing from all of you!
> >
> > Best regards,
> > Sönke
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-317%3A+Add+transparent+data+encryption+functionality
> >
>


-- 
Sönke Liebau
Partner
Tel. +49 179 7940878
OpenCore GmbH & Co. KG - Thomas-Mann-Straße 8 - 22880 Wedel - Germany


Problems with broker upgrade from 1.1.0 to 2.0.0

2018-10-08 Thread Claudia Wegmann
Dear community,

I updated my kafka cluster from version 1.1.0 to 2.0.0 according to the upgrade 
guide for rolling upgrades today. I encountered a problem after starting the 
new broker version. The log is full of
"Found a corrupted index file corresponding to log file 
/data/kafka-logs/resultGatewayDataDispatcher-22/02858226.log due to 
Corrupt time index found, time index file 
(/data/kafka-logs/resultGatewayDataDispatcher-22/02858226.timeindex)
 has non-zero size but the last timestamp is 0 which is less than the first 
timestamp 1538583935827}, recovering segment and rebuilding index files... 
(kafka.log.Log)"

Said recovering takes quite a bit of time.

Now I'm not too keen to do the broker update in the production environment 
because of the relative long processing pause. 
The questions are:
1.) Is there a way to avoid this error and therefore make starting the new 
broker faster?
2.) Does the error imply any data loss?

Any help is apricated 😊
Best,
Claudia


How to manipulate client settings through AdminAPI?

2018-10-08 Thread Will Weber
Hey all,

Working on a tool for Kafka lifecycle management and one of the features
we're keen to try to wrap is user quotas.

This capability definitely exists through sets of scripts included with
Kafka, case and point: https://kafka.apache.org/documentation/#quotas

For context:

> bin/kafka-configs.sh  --zookeeper localhost:2181 --alter --add-config
'producer_byte_rate=1024,consumer_byte_rate=2048,request_percentage=200'
--entity-type users --entity-name user1 --entity-type clients --entity-name
clientA

Updated config for entity: user-principal 'user1', client-id 'clientA'.

Which would set limits on a given client.

Based on this command, it leads me to believe that the
AdminClient#alterConfigs

method
is probably the way to go.

However, I noticed that in the ConfigResource.Type

 enum, there are only Broker, Unknown and Topic options. Naively, I imagine
there to be a corresponding Enum to match the entity-type flag options as
well, but no such luck.

Anyone have recommendations as to how I could go about enforcing quotas
through the admin api?

Mostly just looking for a nudge in the right direction, any help is
appreciated!

Best,

-Will


Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread John Roesler
Hi Bart,

This sounds a bit surprising. Is there any chance you can zip up some logs
so we can see the assignment protocol on the nodes?

Thanks,
-John

On Mon, Oct 8, 2018 at 4:32 AM Bart Vercammen  wrote:

> Hi,
>
> I recently moved some KafkaStreams applications from v0.10.2.1 to v1.1.1
> and now I notice a weird behaviour in the partition assignment.
> When starting 4 instances of my Kafka Streams application (on v1.1.1) I see
> that 17 of the 20 partitions (of a source topic) are assigned to 1 instance
> of the application while the other 3 instances only get 1 partition
> assigned. (previously (on v0.10.2.1) the all got 5 partitions.)
>
> Is this expected behaviour, as I read that quite some improvements were
> done in the partition assignment strategy for Kafka Streams applications?
> If yes, how can I make it so that the partitions are equally devided again
> across all running applications?   It's a bit weird in my opinion as this
> makes scaling the application very hard.
>
> Also, when initially starting with 1 instance of the application, and
> gradually scaling up, the new instances only get 1 partition assigned ...
>
> All my Streams applications use default configuration (more or less),
> running 1 stream-thread.
>
> Any suggestions / enlightenments on this?
> Greets,
> Bart
>


Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread Bart Vercammen
Hi John,

Zipping up some logs from our running Kafka cluster is going to be a bit
difficult.
What I can do is try to reproduce this off-line and capture the logs from
there.

We also had a look in the PartitionAssignor source code (for 1.1.1) and
indeed this behaviour is a bit weird
as from the source code I'd expect equally divided partitions.

Anyway, hopefully I'll be able to reproduce this issue with some simple
unit-test like code.
I'll post the results when I have more info.

Greets,
Bart

On Mon, Oct 8, 2018 at 7:36 PM John Roesler  wrote:

> Hi Bart,
>
> This sounds a bit surprising. Is there any chance you can zip up some logs
> so we can see the assignment protocol on the nodes?
>
> Thanks,
> -John
>
> On Mon, Oct 8, 2018 at 4:32 AM Bart Vercammen  wrote:
>
> > Hi,
> >
> > I recently moved some KafkaStreams applications from v0.10.2.1 to v1.1.1
> > and now I notice a weird behaviour in the partition assignment.
> > When starting 4 instances of my Kafka Streams application (on v1.1.1) I
> see
> > that 17 of the 20 partitions (of a source topic) are assigned to 1
> instance
> > of the application while the other 3 instances only get 1 partition
> > assigned. (previously (on v0.10.2.1) the all got 5 partitions.)
> >
> > Is this expected behaviour, as I read that quite some improvements were
> > done in the partition assignment strategy for Kafka Streams applications?
> > If yes, how can I make it so that the partitions are equally devided
> again
> > across all running applications?   It's a bit weird in my opinion as this
> > makes scaling the application very hard.
> >
> > Also, when initially starting with 1 instance of the application, and
> > gradually scaling up, the new instances only get 1 partition assigned ...
> >
> > All my Streams applications use default configuration (more or less),
> > running 1 stream-thread.
> >
> > Any suggestions / enlightenments on this?
> > Greets,
> > Bart
> >
>


-- 
Mvg,
Bart Vercammen


clouTrix BVBA
+32 486 69 17 68
i...@cloutrix.com


Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread John Roesler
Hi Bart,

I suspected it might not be feasible to just dump your production logs onto
the internet.

A repro would be even better, but I bet it wouldn't show up when you try
and reproduce it. Good luck!

If the repro doesn't turn out, maybe you could just extract the assignment
lines from your logs?

Thanks,
-John

On Mon, Oct 8, 2018 at 1:24 PM Bart Vercammen  wrote:

> Hi John,
>
> Zipping up some logs from our running Kafka cluster is going to be a bit
> difficult.
> What I can do is try to reproduce this off-line and capture the logs from
> there.
>
> We also had a look in the PartitionAssignor source code (for 1.1.1) and
> indeed this behaviour is a bit weird
> as from the source code I'd expect equally divided partitions.
>
> Anyway, hopefully I'll be able to reproduce this issue with some simple
> unit-test like code.
> I'll post the results when I have more info.
>
> Greets,
> Bart
>
> On Mon, Oct 8, 2018 at 7:36 PM John Roesler  wrote:
>
> > Hi Bart,
> >
> > This sounds a bit surprising. Is there any chance you can zip up some
> logs
> > so we can see the assignment protocol on the nodes?
> >
> > Thanks,
> > -John
> >
> > On Mon, Oct 8, 2018 at 4:32 AM Bart Vercammen  wrote:
> >
> > > Hi,
> > >
> > > I recently moved some KafkaStreams applications from v0.10.2.1 to
> v1.1.1
> > > and now I notice a weird behaviour in the partition assignment.
> > > When starting 4 instances of my Kafka Streams application (on v1.1.1) I
> > see
> > > that 17 of the 20 partitions (of a source topic) are assigned to 1
> > instance
> > > of the application while the other 3 instances only get 1 partition
> > > assigned. (previously (on v0.10.2.1) the all got 5 partitions.)
> > >
> > > Is this expected behaviour, as I read that quite some improvements were
> > > done in the partition assignment strategy for Kafka Streams
> applications?
> > > If yes, how can I make it so that the partitions are equally devided
> > again
> > > across all running applications?   It's a bit weird in my opinion as
> this
> > > makes scaling the application very hard.
> > >
> > > Also, when initially starting with 1 instance of the application, and
> > > gradually scaling up, the new instances only get 1 partition assigned
> ...
> > >
> > > All my Streams applications use default configuration (more or less),
> > > running 1 stream-thread.
> > >
> > > Any suggestions / enlightenments on this?
> > > Greets,
> > > Bart
> > >
> >
>
>
> --
> Mvg,
> Bart Vercammen
>
>
> clouTrix BVBA
> +32 486 69 17 68
> i...@cloutrix.com
>


Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread Bart Vercammen
Thanks John,

I'll see what I can do regarding the logs ...
As a side not, our Kafka cluster is running version v1.1.1 in v0.10.2.1 log
format configuration (due to another issue: KAFKA-6000)
But, as said, I'll try to come up with some detailed logs, or a scenario to
reproduce this.

Greets,
Bart

On Mon, Oct 8, 2018 at 8:37 PM John Roesler  wrote:

> Hi Bart,
>
> I suspected it might not be feasible to just dump your production logs onto
> the internet.
>
> A repro would be even better, but I bet it wouldn't show up when you try
> and reproduce it. Good luck!
>
> If the repro doesn't turn out, maybe you could just extract the assignment
> lines from your logs?
>
> Thanks,
> -John
>
> On Mon, Oct 8, 2018 at 1:24 PM Bart Vercammen  wrote:
>
> > Hi John,
> >
> > Zipping up some logs from our running Kafka cluster is going to be a bit
> > difficult.
> > What I can do is try to reproduce this off-line and capture the logs from
> > there.
> >
> > We also had a look in the PartitionAssignor source code (for 1.1.1) and
> > indeed this behaviour is a bit weird
> > as from the source code I'd expect equally divided partitions.
> >
> > Anyway, hopefully I'll be able to reproduce this issue with some simple
> > unit-test like code.
> > I'll post the results when I have more info.
> >
> > Greets,
> > Bart
> >
> > On Mon, Oct 8, 2018 at 7:36 PM John Roesler  wrote:
> >
> > > Hi Bart,
> > >
> > > This sounds a bit surprising. Is there any chance you can zip up some
> > logs
> > > so we can see the assignment protocol on the nodes?
> > >
> > > Thanks,
> > > -John
> > >
> > > On Mon, Oct 8, 2018 at 4:32 AM Bart Vercammen 
> wrote:
> > >
> > > > Hi,
> > > >
> > > > I recently moved some KafkaStreams applications from v0.10.2.1 to
> > v1.1.1
> > > > and now I notice a weird behaviour in the partition assignment.
> > > > When starting 4 instances of my Kafka Streams application (on
> v1.1.1) I
> > > see
> > > > that 17 of the 20 partitions (of a source topic) are assigned to 1
> > > instance
> > > > of the application while the other 3 instances only get 1 partition
> > > > assigned. (previously (on v0.10.2.1) the all got 5 partitions.)
> > > >
> > > > Is this expected behaviour, as I read that quite some improvements
> were
> > > > done in the partition assignment strategy for Kafka Streams
> > applications?
> > > > If yes, how can I make it so that the partitions are equally devided
> > > again
> > > > across all running applications?   It's a bit weird in my opinion as
> > this
> > > > makes scaling the application very hard.
> > > >
> > > > Also, when initially starting with 1 instance of the application, and
> > > > gradually scaling up, the new instances only get 1 partition assigned
> > ...
> > > >
> > > > All my Streams applications use default configuration (more or less),
> > > > running 1 stream-thread.
> > > >
> > > > Any suggestions / enlightenments on this?
> > > > Greets,
> > > > Bart
> > > >
>
>


Re: [KafkaStreams 1.1.1] partition assignment broken?

2018-10-08 Thread Bill Bejeck
Hi Bart,

This is a known issue discovered in version 1.1 -
https://issues.apache.org/jira/browse/KAFKA-7144

This issue has been fixed in Kafka Streams 2.0, any chance you can upgrade
to 2.0?

Thanks,
Bill

On Mon, Oct 8, 2018 at 2:46 PM Bart Vercammen  wrote:

> Thanks John,
>
> I'll see what I can do regarding the logs ...
> As a side not, our Kafka cluster is running version v1.1.1 in v0.10.2.1 log
> format configuration (due to another issue: KAFKA-6000)
> But, as said, I'll try to come up with some detailed logs, or a scenario to
> reproduce this.
>
> Greets,
> Bart
>
> On Mon, Oct 8, 2018 at 8:37 PM John Roesler  wrote:
>
> > Hi Bart,
> >
> > I suspected it might not be feasible to just dump your production logs
> onto
> > the internet.
> >
> > A repro would be even better, but I bet it wouldn't show up when you try
> > and reproduce it. Good luck!
> >
> > If the repro doesn't turn out, maybe you could just extract the
> assignment
> > lines from your logs?
> >
> > Thanks,
> > -John
> >
> > On Mon, Oct 8, 2018 at 1:24 PM Bart Vercammen  wrote:
> >
> > > Hi John,
> > >
> > > Zipping up some logs from our running Kafka cluster is going to be a
> bit
> > > difficult.
> > > What I can do is try to reproduce this off-line and capture the logs
> from
> > > there.
> > >
> > > We also had a look in the PartitionAssignor source code (for 1.1.1) and
> > > indeed this behaviour is a bit weird
> > > as from the source code I'd expect equally divided partitions.
> > >
> > > Anyway, hopefully I'll be able to reproduce this issue with some simple
> > > unit-test like code.
> > > I'll post the results when I have more info.
> > >
> > > Greets,
> > > Bart
> > >
> > > On Mon, Oct 8, 2018 at 7:36 PM John Roesler  wrote:
> > >
> > > > Hi Bart,
> > > >
> > > > This sounds a bit surprising. Is there any chance you can zip up some
> > > logs
> > > > so we can see the assignment protocol on the nodes?
> > > >
> > > > Thanks,
> > > > -John
> > > >
> > > > On Mon, Oct 8, 2018 at 4:32 AM Bart Vercammen 
> > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > I recently moved some KafkaStreams applications from v0.10.2.1 to
> > > v1.1.1
> > > > > and now I notice a weird behaviour in the partition assignment.
> > > > > When starting 4 instances of my Kafka Streams application (on
> > v1.1.1) I
> > > > see
> > > > > that 17 of the 20 partitions (of a source topic) are assigned to 1
> > > > instance
> > > > > of the application while the other 3 instances only get 1 partition
> > > > > assigned. (previously (on v0.10.2.1) the all got 5 partitions.)
> > > > >
> > > > > Is this expected behaviour, as I read that quite some improvements
> > were
> > > > > done in the partition assignment strategy for Kafka Streams
> > > applications?
> > > > > If yes, how can I make it so that the partitions are equally
> devided
> > > > again
> > > > > across all running applications?   It's a bit weird in my opinion
> as
> > > this
> > > > > makes scaling the application very hard.
> > > > >
> > > > > Also, when initially starting with 1 instance of the application,
> and
> > > > > gradually scaling up, the new instances only get 1 partition
> assigned
> > > ...
> > > > >
> > > > > All my Streams applications use default configuration (more or
> less),
> > > > > running 1 stream-thread.
> > > > >
> > > > > Any suggestions / enlightenments on this?
> > > > > Greets,
> > > > > Bart
> > > > >
> >
> >
>


Re: Problems with broker upgrade from 1.1.0 to 2.0.0

2018-10-08 Thread Brett Rann
I'd

* verify you shutdown cleanly when restarting the broker
* test restarting the broker first before upgrading it (it may have been
sitting on a corrupt file for a long time and the issue is unrelated to the
upgrade)

re your questions:

1) probably not, unless it was caused by an unclean shutdown, in which case
shutdown cleanly. :)
2) probably not, since the indexes are rebuilt.

FWIW i've done a bunch of 1.1.0 -> 2.0 upgrades and haven't had this issue.
For sure have seen it in the past though.


On Tue, Oct 9, 2018 at 2:09 AM Claudia Wegmann  wrote:

> Dear community,
>
> I updated my kafka cluster from version 1.1.0 to 2.0.0 according to the
> upgrade guide for rolling upgrades today. I encountered a problem after
> starting the new broker version. The log is full of
> "Found a corrupted index file corresponding to log file
> /data/kafka-logs/resultGatewayDataDispatcher-22/02858226.log
> due to Corrupt time index found, time index file
> (/data/kafka-logs/resultGatewayDataDispatcher-22/02858226.timeindex)
> has non-zero size but the last timestamp is 0 which is less than the first
> timestamp 1538583935827}, recovering segment and rebuilding index files...
> (kafka.log.Log)"
>
> Said recovering takes quite a bit of time.
>
> Now I'm not too keen to do the broker update in the production environment
> because of the relative long processing pause.
> The questions are:
> 1.) Is there a way to avoid this error and therefore make starting the new
> broker faster?
> 2.) Does the error imply any data loss?
>
> Any help is apricated 😊
> Best,
> Claudia
>