That's great news. Will be keeping an eye on the release.
On Mon, Aug 8, 2016 at 10:12 AM Jacob Maes
wrote:
> Hey David,
>
> I think that behavior is meant to prevent an issue on the Kafka 0.8
> Brokers. Samza 10.1 allows compression on log compacted topics, but you'll
> need to make sure you're
Hey David,
I think that behavior is meant to prevent an issue on the Kafka 0.8
Brokers. Samza 10.1 allows compression on log compacted topics, but you'll
need to make sure you're using Kafka 0.9 or higher on the Brokers.
-Jake
On Fri, Aug 5, 2016 at 10:57 PM, David Yu wrote:
> I guess this mig
I guess this might be the problem:
2016-08-06 05:23:23,622 [main ] WARN o.a.s.s.kafka.KafkaSystemFactory$ -
System name 'kafka' is being used as a changelog. Disabling compression
since Kafka does not support compression for log compacted topics.
Is this a 0.8.x.x Kafka producer limitation?
On
Hey David,
If you check your container logs Kafka should print the producer config
> 2016-08-05 19:18:30.134 [main] ProducerConfig [INFO] ProducerConfig values:
> compression.type = gzip
...
If you see the correct compression type in that config, then the Kafka
producer definitely has compressi
I'm reporting back my observations after enabling compression.
Looks like compression is not doing anything. I'm still seeing
"compression-rate-avg=1.0" and the same "record-size-avg" from JMX
"kafka.producer" metrics.
I did set the following:
systems.kafka.producer.compression.type=snappy
Am I
Great. Thx.
On Wed, Aug 3, 2016 at 1:42 PM Jacob Maes wrote:
> Hey David,
>
> what gets written to the changelog topic
>
> The changelog gets the same value as the store, which is the serialized
> form of the key and value. The serdes for the store are configured with the
> properties:
> stores.
Hey David,
what gets written to the changelog topic
The changelog gets the same value as the store, which is the serialized
form of the key and value. The serdes for the store are configured with the
properties:
stores.store-name.key.serde
stores.store-name.msg.serde
If I want to compress the ch
I'm trying to understand what gets written to the changelog topic. Is it
just the serialized value of the particular state store entry? If I want to
compress the changelog topic, do I enable that from the producer?
The reason I'm asking is that, we are seeing producer throughput issues and
suspect
changelog topic. The # of
>> partitions
>> > in
>> > > that changelog topic is equal to the # of tasks. Each task's K-V store
>> > will
>> > > be mapped to a particular partition of that changelog topic. This
>> mapping
>> > >
on't want this k-v changelog topic to keep growing. So,
> > > people configure it with some expiration. The expiration can either be:
> > > 1. Time retention: Records older than the retention are purged.
> > > 2. Compaction: Newer key-values will over-write older keys and o
7;m not sure if offsets are always monotonically increasing in Kafka or
> > could change after a compaction/ a time based retention kicks in for the
> > topic partition.
> >
> >
> >
> >
> >
> > On Sat, Jun 11, 2016 at 11:53 PM, David Yu
> > wrote:
&
on kicks in for the
> topic partition.
>
>
>
>
>
> On Sat, Jun 11, 2016 at 11:53 PM, David Yu
> wrote:
>
> > My understanding of store changelog is that, each task writes store
> changes
> > to a particular changelog partition for that task. (Does that mean t
cent value is retained.
I'm not sure if offsets are always monotonically increasing in Kafka or
could change after a compaction/ a time based retention kicks in for the
topic partition.
On Sat, Jun 11, 2016 at 11:53 PM, David Yu wrote:
> My understanding of store changelog is that
My understanding of store changelog is that, each task writes store changes
to a particular changelog partition for that task. (Does that mean the
changelog keys are task names?)
One thing that confuses me is that, the last offsets of some changelog
partitions do not move. I'm using the
Ah that makes sense now thanks Chris, re-reading that page it is clear. I
think what confused me is this section from the configuration documentation
for `stores.store-name.changelog`:
"... Any output stream can be used as changelog, but you must ensure that
only one job ever writes to a given cha
Also documented here:
http://samza.apache.org/learn/documentation/0.9/container/state-management.html
Check the "Local state in Samza" section - the diagram (and the
description) explains this clearly.
On Thu, Apr 2, 2015 at 10:36 AM, Chris Riccomini
wrote:
> Hey Dan,
>
> I think you might have
Hey Dan,
I think you might have a misunderstanding in how changelogs work with
Samza. Suppose you have a job with two tasks, and a single kv-store is
configured with a changelog attached. The changelog, in Kafka, will have
two partitions. Each task will use one partition of the changelog topic.
Yo
Hi all,
We're just starting out using Samza to process streams we've already got in
Kafka. Some of the jobs we've written are using the per task KV store which
are being persisted to a changelog topic in Kafka. As you need a different
changelog topic per task we are wondering how people are dealin
18 matches
Mail list logo