Re: More partitions => less throughput?

2019-11-30 Thread Tom Brown
I think the number of partitions needs to be tuned to the size of the cluster; 64 partitions on what is essentially a single box seems high. Do you know what hardware you will be deploying on in production? Can you run your benchmark on that instead of a vm? —Tom On Thursday, November 28, 2019, C

Re: Kafka encryption

2016-06-06 Thread Tom Brown
How would it be possible to encrypt an entire batch? My understanding is that the Kafka server needs to know the boundaries of each message. (E.g. The server decompresses compressed message sets and re-compresses individual messages). Given that precedent, how could the server properly separate th

Re: Encryption at Rest

2016-05-02 Thread Tom Brown
I'm trying to understand your use-case for encrypted data. Does it need to be encrypted only over the wire? This can be accomplished using TLS encryption (v0.9.0.0+). See https://issues.apache.org/jira/browse/KAFKA-1690 Does it need to be encrypted only when at rest? This can be accomplished usin

Re: Consumers disappearing form __consumer_offsets

2016-04-11 Thread Tom Brown
Related: Can the __consumer_offsets topic be configured to retain offsets forever no matter how the rest of the server is configured? --Tom On Mon, Apr 11, 2016 at 3:19 PM, Morellato, Wanny < wanny.morell...@concur.com> wrote: > Thanks James, That was exactly what I was looking for. > > Wanny >

Datacenter to datacenter over the open internet

2015-10-06 Thread Tom Brown
Hello, How do you consume a kafka topic from a remote location without a dedicated connection? How do you protect the server? The setup: data streams into our datacenter. We process it, and publish it to a kafka cluster. The consumer is located in a different datacenter with no direct connection.

Re: Handling un-decodable messages (Kafka 0.8.0)

2014-06-10 Thread Tom Brown
Another way would be to have your custom decoder return an object that can be recognized as an error. We have a decoder that splits binary data into a series of records. If any part of the binary data is corrupt, the decoder can be configured to either throw an exception or add an "error record" t

Re: New Consumer API discussion

2014-02-13 Thread Tom Brown
Conceptually, do the position methods only apply to topics you've subscribed to, or do they apply to all topics in the cluster? E.g., could I retrieve or set the committed position of any partition? The positive use case for having access to all partition information would be to setup an active m

Re: C++ Producer => Broker => Java Consumer?

2014-01-31 Thread Tom Brown
The C++ program writes bytes to kafka, and java reads bytes from kafka. Is there something special about the way the messages are being serialized in C++? --Tom On Fri, Jan 31, 2014 at 2:36 PM, Philip O'Toole wrote: > Is this a Kafka C++ lib you wrote yourself, or some open-source library? >

Re: New Producer Public API

2014-01-31 Thread Tom Brown
Regarding partitioning APIs, I don't think there is not a common subset of information that is required for all strategies. Instead of modifying the core API to easily support all of the various partitioning strategies, offer the most common ones as libraries they can build into their own data pipe

Re: New Producer Public API

2014-01-31 Thread Tom Brown
The trouble with callbacks, IMHO, is determining the thread in which they will be executed. Since the IO thread is usually the thread that knows when the operation is complete, it's easiest to execute that callback within the IO thread. This can lead the IO thread to spend all its time on callbacks

Re: How to design a robust producer?

2014-01-30 Thread Tom Brown
data to a specific topic in your example? Through which client? > > Le 30 janv. 2014 à 15:16, Tom Brown a écrit : > > > Why go with a fancy multithreaded producer architecture? Why not rely on > a > > simple python/perl/whatever implementation and let a scalable web server > > handle the threading issues? > >

Re: How to design a robust producer?

2014-01-30 Thread Tom Brown
Why go with a fancy multithreaded producer architecture? Why not rely on a simple python/perl/whatever implementation and let a scalable web server handle the threading issues?

Re: New Producer Public API

2014-01-29 Thread Tom Brown
integer and let the user figure > out > > > the > > > > mapping from partition_key -> partition_id using the getCluster() > API? > > > If I > > > > understand correctly, this is similar to what you suggested as part > of > > > > option 1A. I like this app

Re: New Producer Public API

2014-01-29 Thread Tom Brown
t; > understand correctly, this is similar to what you suggested as part > of > > > > option 1A. I like this approach since it maintains the simplicity of > > APIs > > > > by allowing us to deal with bytes and does not compromise performance > > in > &g

Re: New Producer Public API

2014-01-29 Thread Tom Brown
the custom partitioning case. > > > > Thanks, > > Neha > > > > > > > > On Tue, Jan 28, 2014 at 5:42 PM, Jay Kreps wrote: > > > > > Hey Tom, > > > > > > That sounds cool. How did you end up handling parallel I/O if you wrap &

Re: New Producer Public API

2014-01-28 Thread Tom Brown
I implemented a 0.7 client in pure java, and its API very closely resembled this. (When multiple people independently engineer the same solution, it's probably good... right?). However, there were a few architectural differences with my client: 1. The basic client itself was just an asynchronous l

Re: Partial Message Read by Consumer

2013-12-10 Thread Tom Brown
Having a partial message transfer over the network is the design of Kafka 0.7.x (I can't speak to 0.8.x, though it may still be). When the request is made, you tell the server the partition number, the byte offset into that partition, and the size of response that you want. The server finds that o

Re: Consuming "backwards"?

2013-12-06 Thread Tom Brown
Do you mean you want to start from the most recent data and go backwards to the oldest data, or that you want to start with old data and consume forwards? If the former, it would be difficult or impossible in 0.7.x, but I think doable in 0.8.x. (They added some sort of message index). If the latte

Updated kafka client (producer and consumer)

2013-12-05 Thread Tom Brown
In our environment we use currently use Kafka 0.7.1. The core features I am looking for in a client are this: 1. Provide confirmation of produce requests (or notification of disconnection during requests). 2. Uses asynchronous IO so that: A. Multiple ops can be queued/in-flight at once. B. Fe

Cross version kafka wire-format compatibility

2013-11-25 Thread Tom Brown
This document describes the wire-format used by kafka at some point during 0.7.x. Is this still valid for 0.8.x? https://cwiki.apache.org/confluence/display/KAFKA/Writing+a+Driver+for+Kafka If not, is there another document that describes the new format? Thanks in advance! --Tom

Re: Best partition configuration

2013-08-21 Thread Tom Brown
Philip, How many topics per broker (just one?) And what is the read/write profile of your setup? --Tom On Wed, Aug 21, 2013 at 12:24 PM, Philip O'Toole wrote: > On Wed, Aug 21, 2013 at 10:06:50AM -0400, Tom Brown wrote: > > I have not been able to find reliable advice rega

Best partition configuration

2013-08-21 Thread Tom Brown
I have not been able to find reliable advice regarding how many partitions should exist on a single broker. How many partitions have you used, and what kind of throughput have you seen? Thanks in advance! --Tom

Re: kafka multi threaded consumer with very low throughput

2013-07-08 Thread Tom Brown
What is the size of each message? --Tom On Mon, Jul 8, 2013 at 9:04 AM, Anurup Raveendran < anurup.raveend...@fluturasolutions.com> wrote: > I have 2 kafka brokers running on two systems with the same configuration > > CPU - Dual Core > RAM - 4 GB > > I'm trying to benchmark my kafka setup > Nu

Re: Not balancing across multiple brokers

2013-04-12 Thread Tom Brown
PM, Neha Narkhede wrote: > Do you use a VIP or zookeeper for producer side load balancing ? In > other words, what are the values you override for "broker.list" and > "zk.connect" in the producer config ? > > Thanks, > Neha > > On Fri, Apr 12, 2013 at 12:1

Not balancing across multiple brokers

2013-04-12 Thread Tom Brown
We have recently setup a new kafka (0.7.1) cluster with two brokers. Each topic has 2 partitions per server. We have a two processes that that write to the cluster using the class: kafka.javaapi.producer.Producer.Producer. The problem is that the first process only writes to the first broker. The

Re: Kafka log retentions

2013-04-01 Thread Tom Brown
I've often wondered about what it would take to be able to overwrite a specific offset in a partition (it could be very useful for transaction rollbacks, message deletions, etc). Unfortunately, I don't think that feature currently exists. --Tom On Mon, Apr 1, 2013 at 11:14 PM, Pankaj Misra wrote

Re: How do you keep track of offset in a partition

2013-01-28 Thread Tom Brown
Since offsets in Kafka 0.7x are just byte counts, you cannot know the number of messages remaining to be processed (subtract your consumers offsets from each partitions end offset). However, you can know the number of bytes remaining. Knowing the average message size, you can use that to make a rou

Re: Proper use of ConsumerConnector

2012-12-21 Thread Tom Brown
It seems that a common thread is that while ConsumerConnector works well for the standard case, it just doesn't work for any case where manual offset management (explicit checkpoints, rollbacks, etc) is required. If any Kafka devs are looking for a way to improve it, I think modifying it to be mor

Re: Proper use of ConsumerConnector

2012-12-21 Thread Tom Brown
Does the ConsumerConnector keep track of the offsets of data downloaded from the server (and queued for consumption by the end user of the API), or does it keep track of the actual offset that has been consumed by the end user? --Tom On Fri, Dec 21, 2012 at 10:37 AM, Neha Narkhede wrote: >> But

Re: Proper use of ConsumerConnector

2012-12-20 Thread Tom Brown
In order to support rollbacks and checkpoints, there would have to be a way to both supply partition offsets to the consumer before reading, as well as retrieve partition offsets from them consumer once reading is complete. >From what I've read here, it appears that neither the ConsumerConnector n

Proper use of ConsumerConnector

2012-12-19 Thread Tom Brown
I wish to use either the ConsumerConnector or the SimpleConsumer to read messages from all partitions across multiple brokers using a fixed number of threads (hopefully leveraging asynchronous IO for high performance). I know that the ConsumerConnector sounds like this, but the documentation was n

Re: Fetch messages since a specific time?

2012-12-17 Thread Tom Brown
Each message does not have a time stamp. Groups of messages (I think the default is around 500mb) are stored in individual files, and the time stamp parameter will find the offset at the beginning of the file that has that time stamp-- not really helpful for your use case. The accepted solution is