RE: Broker Interceptors

2019-12-04 Thread Thomas Aley
Thanks for the responses. I did worry about the challenge of exposing a 
vast number of internal classes with general interceptor framework. A less 
general solution more along the lines of the producer/consumer 
interceptors on the client would satisfy the majority of use cases. If we 
are smart, we should be able to come up with a pattern that could be 
extended further in future if the community sees the demand.

Looking through the discussion thread for KIP-388, I see a lot of good 
points to consider and I intend to dive further into this.


Tom Aley
thomas.a...@ibm.com



From:   Ismael Juma 
To: Kafka Users 
Cc: dev 
Date:   03/12/2019 16:12
Subject:[EXTERNAL] Re: Broker Interceptors



The main challenge is doing this without exposing a bunch of internal
classes. I haven't seen a proposal that handles that aspect well so far.

Ismael

On Tue, Dec 3, 2019 at 7:21 AM Sönke Liebau
 wrote:

> Hi Thomas,
>
> I think that idea is worth looking at. As you say, if no interceptor is
> configured then the performance overhead should be negligible. Basically 
it
> is then up to the user to decide if he wants tomtake the performance 
hit.
> We should make sure to think about monitoring capabilities like time 
spent
> in the interceptor for records etc.
>
> The most obvious use case I think is server side schema validation, 
which
> Confluent are also offering as part of their commercial product, but 
other
> ideas come to mind as well.
>
> Best regards,
> Sönke
>
> Thomas Aley  schrieb am Di., 3. Dez. 2019, 10:45:
>
> > Hi M. Manna,
> >
> > Thank you for your feedback, any and all thoughts on this are 
appreciated
> > from the community.
> >
> > I think it is important to distinguish that there are two parts to 
this.
> > One would be a server side interceptor framework and the other would 
be
> > the interceptor implementations themselves.
> >
> > The idea would be that the Interceptor framework manifests as a plug
> point
> > in the request/response paths that by itself has negligible 
performance
> > impact as without an interceptor registered in the framework it is
> > essentially a no-op. This way the out-the-box behavior of the Kafka
> broker
> > remains essentially unchanged, it is only if the cluster administrator
> > registers an interceptor into the framework that the path of a record 
is
> > intercepted. This is much like the already accepted and implemented
> client
> > interceptors - the capability exists and it is an opt-in feature.
> >
> > As with the client interceptors and indeed interception in general, 
the
> > interceptor implementations need to be thoughtfully crafted to ensure
> > minimal performance impact. Yes the interceptor framework could tap 
into
> > nearly everything but would only be tapping into the subset of APIs 
that
> > the user wishes to intercept for their use case.
> >
> > Tom Aley
> > thomas.a...@ibm.com
> >
> >
> >
> > From:   "M. Manna" 
> > To: Kafka Users 
> > Cc: d...@kafka.apache.org
> > Date:   02/12/2019 11:31
> > Subject:[EXTERNAL] Re: Broker Interceptors
> >
> >
> >
> > Hi Tom,
> >
> > On Mon, 2 Dec 2019 at 09:41, Thomas Aley  wrote:
> >
> > > Hi Kafka community,
> > >
> > > I am hoping to get some feedback and thoughts about broker
> interceptors.
> > >
> > > KIP-42 Added Producer and Consumer interceptors which have provided
> > Kafka
> > > users the ability to collect client side metrics and trace the path 
of
> > > individual messages end-to-end.
> > >
> > > This KIP also mentioned "Adding message interceptor on the broker 
makes
> > a
> > > lot of sense, and will add more detail to monitoring. However, the
> > > proposal is to do it later in a separate KIP".
> > >
> > > One of the motivations for leading with client interceptors was to 
gain
> > > experience and see how useable they are before tackling the server 
side
> > > implementation which would ultimately "allow us to have a more
> > > complete/detailed message monitoring".
> > >
> > > Broker interceptors could also provide more value than just more
> > complete
> > > and detailed monitoring such as server side schema validation, so I 
am
> > > curious to learn if anyone in the community has progressed this 
work;
> > has
> > > ideas about other potential server side interceptor uses or has
> actually
> > > implemented something similar.
> > >
> >
> >  I personally feel that the cost he

RE: Broker Interceptors

2019-12-03 Thread Thomas Aley
Hi M. Manna,

Thank you for your feedback, any and all thoughts on this are appreciated 
from the community.

I think it is important to distinguish that there are two parts to this. 
One would be a server side interceptor framework and the other would be 
the interceptor implementations themselves.

The idea would be that the Interceptor framework manifests as a plug point 
in the request/response paths that by itself has negligible performance 
impact as without an interceptor registered in the framework it is 
essentially a no-op. This way the out-the-box behavior of the Kafka broker 
remains essentially unchanged, it is only if the cluster administrator 
registers an interceptor into the framework that the path of a record is 
intercepted. This is much like the already accepted and implemented client 
interceptors - the capability exists and it is an opt-in feature.

As with the client interceptors and indeed interception in general, the 
interceptor implementations need to be thoughtfully crafted to ensure 
minimal performance impact. Yes the interceptor framework could tap into 
nearly everything but would only be tapping into the subset of APIs that 
the user wishes to intercept for their use case. 

Tom Aley
thomas.a...@ibm.com



From:   "M. Manna" 
To: Kafka Users 
Cc: d...@kafka.apache.org
Date:   02/12/2019 11:31
Subject:[EXTERNAL] Re: Broker Interceptors



Hi Tom,

On Mon, 2 Dec 2019 at 09:41, Thomas Aley  wrote:

> Hi Kafka community,
>
> I am hoping to get some feedback and thoughts about broker interceptors.
>
> KIP-42 Added Producer and Consumer interceptors which have provided 
Kafka
> users the ability to collect client side metrics and trace the path of
> individual messages end-to-end.
>
> This KIP also mentioned "Adding message interceptor on the broker makes 
a
> lot of sense, and will add more detail to monitoring. However, the
> proposal is to do it later in a separate KIP".
>
> One of the motivations for leading with client interceptors was to gain
> experience and see how useable they are before tackling the server side
> implementation which would ultimately "allow us to have a more
> complete/detailed message monitoring".
>
> Broker interceptors could also provide more value than just more 
complete
> and detailed monitoring such as server side schema validation, so I am
> curious to learn if anyone in the community has progressed this work; 
has
> ideas about other potential server side interceptor uses or has actually
> implemented something similar.
>

 I personally feel that the cost here is the impact on performance. If I 
am
right, this interceptor is going to tap into nearly everything. If you 
have
strong guarantee (min.in.sync.replicas = N-1) then this may incur some
delay (and let's not forget inter broker comms protection by TLS config).
This may not be desirable for some systems. That said, it would be good to
know what others think about this.

Thanks,

>
> Regards,
>
> Tom Aley
> thomas.a...@ibm.com
> 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
>
>



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



Broker Interceptors

2019-12-02 Thread Thomas Aley
Hi Kafka community,

I am hoping to get some feedback and thoughts about broker interceptors.

KIP-42 Added Producer and Consumer interceptors which have provided Kafka 
users the ability to collect client side metrics and trace the path of 
individual messages end-to-end.

This KIP also mentioned "Adding message interceptor on the broker makes a 
lot of sense, and will add more detail to monitoring. However, the 
proposal is to do it later in a separate KIP".

One of the motivations for leading with client interceptors was to gain 
experience and see how useable they are before tackling the server side 
implementation which would ultimately "allow us to have a more 
complete/detailed message monitoring".

Broker interceptors could also provide more value than just more complete 
and detailed monitoring such as server side schema validation, so I am 
curious to learn if anyone in the community has progressed this work; has 
ideas about other potential server side interceptor uses or has actually 
implemented something similar.

Regards,

Tom Aley
thomas.a...@ibm.com
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



Re: Kafka logs are getting deleted too soon

2019-07-17 Thread Thomas Aley
Hi Sachin,

Try adding --from-beginning to your console consumer to view the 
historically produced data. By default the console consumer starts from 
the last offset.

Tom Aley
thomas.a...@ibm.com



From:   Sachin Nikumbh 
To: Kafka Users 
Date:   17/07/2019 16:01
Subject:[EXTERNAL] Kafka logs are getting deleted too soon



Hi all,
I have ~ 96GB of data in files that I am trying to get into a Kafka 
cluster. I have ~ 11000 keys for the data and I have created 15 partitions 
for my topic. While my producer is dumping data in Kafka, I have a console 
consumer that shows me that kafka is getting the data. The producer runs 
for a few hours before it is done. However, at this point, when I run the 
console consumer, it does not fetch any data. If I look at the logs 
directory, .log files for all the partitions are of 0 byte size. 
If I am not wrong, the default value for log.retention.bytes is -1 which 
means there is no size limit for the logs/partition. I do want to make 
sure that the value for this setting is per partition. Given that the 
default time based retention is 7 days, I am failing to understand why the 
logs are getting deleted. The other thing that confuses me is that when I 
use kafka.tools.GetOffsetShell, it shows me large enough values for all 
the 15 partitions for offsets.
Can someone please help me understand why I don't see logs and why 
is kafka.tools.GetOffsetShell making me believe there is data.
ThanksSachin


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



Re: Kafka behind NAT

2018-05-22 Thread Thomas Aley
Try setting:

advertised.listeners=EXTERNAL://:9093,INTERNAL://:9092
inter.broker.listener.name=INTERNAL
listener.security.protocol.map=EXTERNAL:PLAINTEXT,INTERNAL:PLAINTEXT

Then you should be able to use :9093 as your bootstrap.servers 
from outside the network or :9092 from inside.

Obviously the EXTERNAL listener should be EXTERNAL:SSL in production.

Hope this helps, 

Tom Aley
thomas.a...@ibm.com



From:   "周正虎" 
To: users@kafka.apache.org
Date:   21/05/2018 23:59
Subject:Kafka behind NAT



We have kafka behind NAT with *only one broker*.
Let say we have internal (A) and external (B) network.

When we try to reach the broker from external network (we use
bootstrap.servers parameter set to B address) then what is obvious the
broker responds with internal network's address (A) which is not 
resolvable
in external network. We cannot set advertised.listeners to external
network's address because the broker is also used from internal network.

I hope that somebody dealt with simillar problem.
Thanks for any help.


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



Re: Hardware Guidance

2018-03-01 Thread Thomas Aley
Hi Adrien,

Without asking the author directly I can't give the exact answer but I 
would interpret that as per broker. Kafka will make use of as much 
hardware as you give it so it's not uncommon to see many CPU cores and 
lots or RAM per broker. That being said it's completely down to your use 
case how much hardware you would require. 

Tom Aley
thomas.a...@ibm.com



From:   adrien ruffie 
To: "users@kafka.apache.org" 
Date:   01/03/2018 17:09
Subject:Hardware Guidance



Hi all,


on the slide 5 in the following link:

https://urldefense.proofpoint.com/v2/url?u=https-3A__fr.slideshare.net_HadoopSummit_apache-2Dkafka-2Dbest-2Dpractices_1&d=DwIFAw&c=jf_iaSHvJObTbx-siA1ZOg&r=HAGsP00blUVdJLMAhrBgwP8nKbGaKIkfoe2NJdvKRM0&m=htK1S8vK62xmNDXSuIRHlIB_mR9GYbKv9C4yCA4XYw4&s=s2LkKpG3bZAYZFoaqdXPQ7cZGWL7EUl5aqJ8Qkm-0W4&e=




The "Memory" mentions that "24GB+ (for small) and 64GB+ (for large)" Kafka 
Brokers

but is it 24 or 64 GB spread over all brokers ? Or 24 GB for example for 
each broker ?


Thank you very much,


and best regards,


Adrien



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


Re: When a broker down, Producer LOST messages!

2018-03-01 Thread Thomas Aley
Hi,

The log is not attached. I'm assuming your topic has a replication factor 
greater than 1 so that it is available from another Broker if the 
partition leader fails. Try adding 

props.put("acks", "all");

to your producer and run your experiment again. If you configured your 
topic to have --replication-factor 3 and your brokers (or topic itself) is 
configured with min.insync.replicas=2 for example then your producer will 
require acknowledgment of receipt of each message from 2 of your 3 brokers 
with the 'acks=all' property in use making your topic resilient. 

Hope this helps,

Tom Aley
thomas.a...@ibm.com



From:   "许志峰" 
To: users@kafka.apache.org
Date:   01/03/2018 08:59
Subject:When a broker down, Producer LOST messages!



Hi all,


I have a kafka cluster with 3 nodes: node1, node2, node3

kafka version is 0.8.2.1, which I can not change!

A Producer writes msg to kafka, its code framework is like this in 
pseudo-code:
Properties props = new Properties();
props.put("bootstrap.servers", "node1:9092,node2:9092,node3:9092");
props.put(key serializer and value serializer);

for(i = 0; i < 10; ++i){
producer = new Producer(props);
msg = "this is msg " + i;
producer.send(msg);
producer.close()
}

After the first 4 messages are send successfully, I killed broker on 
node1.  the 5th and 6th messages are LOST.

Producer first get the broker list from the PropertyConfig, i.e. [node1, 
node2, node3], then producer choose one broker, connect with it and get 
METADATA from it. 

I heard that when one broker in the list is unavailable, the kafka client 
will change to another 

But in my case, if the broker choose node1, which is already dead, it will 
get a Fetch MetaData Timeout Exception and STOPPED! msg is not writed into 
Kafka.  

   
Attached is the complete Log. you can only focus on the colorful lines.  

you can see that, I wrote 10 msgs to Kafka, the first 4 succeed, when I 
kill one broker, msg5 and msg6 are LOST, because the choose NODE1, 
msg7,8,9,10 are succeed because they did not choose node1.

I checkout the Kafka source codes and get nothing.

Do anybody know the reason?  where are the related classes/functions 
located in the source code?

Any clue will be appreciated!





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



Re: broker properties explanations

2018-02-21 Thread Thomas Aley
Hi Adrien,

log.dirs exists to facilitate multiple data directories which allows more 
than one disk to be used without the need for RAID. This increases 
throughput but beware of naive load balancing that may fill up one disk 
way before another.

When log.flush.interval.ms is null the log.flush.interval.messages 
property is not used. With default settings, messages are written to disk 
immediately. 

Hope this helps.

Tom Aley
thomas.a...@ibm.com



From:   adrien ruffie 
To: "users@kafka.apache.org" 
Date:   20/02/2018 20:46
Subject:broker properties explanations



Hello all,

after reading several properties in Kafka documentations, I asked mysleft 
some questions ...


these 2 following options are available:

log.dir The directory in which the log data is kept (supplemental for 
log.dirs property)string  /tmp/kafka-logs high
log.dirsThe directories in which the log data is kept. If not set, 
the value in log.dir is used
But in fact, if is the same thing, why only "log.dirs" isn't kept ? What's 
difference in usage ?


Also I noticed that the "data" of logs partition and also the 
"application" logs of Kafka are written in the same directory.

Is a mistake of my part ? Because it's very strange for me to log error, 
debug, warn application message in the same location of my data ...



After that I don't very understand why the log.flush.interval.messages 
have a so big default value ???

log.flush.interval.messages The number of messages accumulated on a 
log partition before messages are flushed to disk   long 
9223372036854775807
And the log.flush.interval.ms is by default null ... ?

It means that until there are so many messages (9223372036854775807) in my 
topics, they will not be flushed to disk ? It can be very long for a 
default value 


Best regards,


Adrien



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