I think you can do this now by using a custom partitioner, no?
https://kafka.apache.org/0110/javadoc/org/apache/kafka/clients/producer/Partitioner.html
-Jay
On Mon, Oct 2, 2017 at 6:29 AM Michal Michalski
wrote:
> Hi,
>
> TL;DR: I'd love to be able to make log
Hey Chris,
I heard a similar complaint from a few people. I am quite ignorant about
event sourcing and don't feel I understand the relationship fully but I am
interested in understanding a little better what you are saying.
I think we see the world this way:
1. You store the log of primary
I suspect this is on Linux right?
The way Linux works is it uses a percent of memory to buffer new writes, at
a certain point it thinks it has too much buffered data and it gives high
priority to writing that out. The good news about this is that the writes
are very linear, well layed out, and
+1
On Sat, May 27, 2017 at 11:04 AM, Gwen Shapira wrote:
> Thanks Vahid,
>
> Do you mind if we leave the command-line out of scope for this?
>
> I can see why adding confirmations, options to bypass confirmations, etc
> would be an improvement. However, I've seen no
I think the question is when do you actually *want* processing time
semantics? There are definitely times when its safe to assume the two are
close enough that a little lossiness doesn't matter much but it is pretty
hard to make assumptions about when the processing time is and has been
hard for
I think the question is when do you actually *want* processing time
semantics? There are definitely times when its safe to assume the two are
close enough that a little lossiness doesn't matter much but it is pretty
hard to make assumptions about when the processing time is and has been
hard for
;>> computation: scala.util.Try[Int] =
>>>>>> Failure(java.lang.ArithmeticException: / by zero)
>>>>>>
>>>>>> scala> computation.getOrElse(42)
>>>>>> res2: Int = 42
>>>>>>
Hey Eno,
I think this makes sense. I do think people who spend time running
production stream processing systems will, over time, end up strongly
preferring the current behavior of failing and fixing the root problem
rather than skipping, but we don't need to force this on people as long as
the
I think setting max.block.ms=0 does what you want.
-Jay
On Sat, May 27, 2017 at 12:40 PM, Abhimanyu Nagrath <
abhimanyunagr...@gmail.com> wrote:
> HI Hans,
>
> What exactly I meant by asynchronous is that when my Kafka broker is down
> and I am trying to produce the message . It is getting
This is great!
-Jay
On Sat, May 27, 2017 at 12:47 PM, Michal Borowiecki <
michal.borowie...@openbet.com> wrote:
> Hi all,
>
> I've updated the wiki page with a draft pattern for consecutively growing
> time-windowed aggregations which was discussed some time ago on this
> mailing list.
>
> I'm
+1
On Tue, May 9, 2017 at 3:41 PM BigData dev wrote:
> Hi, Everyone,
>
> Since this is a relatively simple change, I would like to start the voting
> process for KIP-156: Add option "dry run" to Streams application reset tool
>
>
Congrats Rajini!
On Mon, Apr 24, 2017 at 2:06 PM, Gwen Shapira wrote:
> The PMC for Apache Kafka has invited Rajini Sivaram as a committer and we
> are pleased to announce that she has accepted!
>
> Rajini contributed 83 patches, 8 KIPs (all security and quota
> improvements)
gt;>
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> On 3/14/17 3:36 AM, Michael Noll wrote:
> > > >>>>>> I see Jay's point, and I agree with much of it -- notably about
> >
cope for this KIP. KIP-120 has the focus on removing leaking
> internal APIs and do some cleanup how our API reflects some concepts.
>
> However, I added your idea to API discussion Wiki page and we take if
> from there:
> https://cwiki.apache.org/confluence/display/KAFKA/
> K
Two things:
1. This is a minor thing but the proposed new name for KStreamBuilder
is StreamsTopologyBuilder. I actually think we should not put topology in
the name as topology is not a concept you need to understand at the
kstreams layer right now. I'd think of three categories of
+1
Nice improvement.
-Jay
On Tue, Feb 14, 2017 at 1:22 PM, Steven Schlansker <
sschlans...@opentable.com> wrote:
> Hi, it looks like I have 2 of the 3 minimum votes, can a third voter
> please consider this KIP?
> Thanks.
>
> (PS - new revision on GitHub PR with hopefully the last round of
>
Congrats Grant!
-Jay
On Wed, Jan 11, 2017 at 11:51 AM, Gwen Shapira wrote:
> The PMC for Apache Kafka has invited Grant Henke to join as a
> committer and we are pleased to announce that he has accepted!
>
> Grant contributed 88 patches, 90 code reviews, countless great
>
+1
On Wed, Jan 11, 2017 at 10:56 AM, Ben Stopford wrote:
> Looks like there was a good consensus on the discuss thread for KIP-106 so
> lets move to a vote.
>
> Please chime in if you would like to change the default for
> unclean.leader.election.enabled from true to false.
>
Good question! Here's my understanding.
The streams API has a config num.standby.replicas. If this value is set to
0, the default, then the local state will have to be recreated by
re-reading the relevant Kafka partition and replaying that into the state
store, and as you point out this will take
I think the most common cause of rebalancing is still GC that exceeds the
consumer liveness timeout you've configured. Might be worth enabling GC
logging in java and then checking the pause times. If they exceed the
timeout you have for liveness then you will detect that as a process
failure and
-1
I think the REST server for Kafka that already exists is quite good and
getting contributions. Moving this into the core project doesn't solve a
problem that I see.
-Jay
On Tue, Oct 25, 2016 at 2:16 PM, Harsha Chintalapani
wrote:
> Hi All,
>We are proposing to
Can you describe the behavior you saw that you didn't like?
-Jay
On Mon, Aug 22, 2016 at 12:24 AM, Jan Filipiak
wrote:
> Hello everyone,
>
> I stumbled across KIP-33 and the time based index, while briefly checking
> the wiki and commits, I fail to find a way to opt
I think you may get this for free as Kafka Streams attempts to align
consumption across different topics/partitions by the timestamp in the
messages. So in a case where you are starting a job fresh and it has a
database changelog to consume and a event stream to consume, it will
attempt to keep
Even though I'm not aware of anyone working on it, I think we'd definitely
be open to it if someone wants to take a swing at it.
-Jay
On Sun, Jun 26, 2016 at 9:13 AM, Alex Glikson wrote:
> Hi all,
>
> I wonder whether there are plans to implement Apache Beam backend based
Can you sanity check this with the end-to-end latency test that ships with
Kafka in the tools package?
https://apache.googlesource.com/kafka/+/1769642bb779921267bd57d3d338591dbdf33842/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala
On Saturday, June 25, 2016, Kafka
un, Drausin Wulsin, Duncan Sands, Dustin Cote,
> Eamon Zhang, edoardo, Edward Ribeiro, Eno Thereska, Ewen
> Cheslack-Postava, Flavio Junqueira, Francois Visconte, Frank Scholten,
> Gabriel Zhang, gaob13, Geoff Anderson, glikson, Grant Henke, Greg
> Fodor, Guozhang Wang, Gwen Shapira, I
Is it possible to make the error message give more an explanation?
-Jay
On Wed, May 4, 2016 at 8:46 AM, Matthias J. Sax
wrote:
> Hi,
>
> I am still new to Kafka Streams by myself, but from my understanding if
> you change the key, your partitioning changes, ie, is not
This is great!
-Jay
On Fri, Apr 22, 2016 at 2:28 PM, Surendra , Manchikanti <
surendra.manchika...@gmail.com> wrote:
> Hi,
>
> I have implemented KafkaConnector for Solr, Please find the below github
> link.
>
> https://github.com/msurendra/kafka-connect-solr
>
> The initial release having
Another way to think about this is that the producer allows you to PUSH
data into Kafka and the consumer allows you to PULL data out. This is what
you need to write an application.
However for an existing data system you need the opposite you need to PULL
data into Kafka from the system or PUSH
There isn't a jdbc sink yet, though that is actually a very useful and easy
connector to write. I think it can be pretty efficient as long as it uses
prepared statements and batch insert...
-Jay
On Fri, Apr 1, 2016 at 6:06 AM, Kavitha Veluri
wrote:
> Thank you for
If you hard kill the broker when it restarts it doesn't know the
status of it's on-disk files, it will need to run though the last log
segment to validate the checksums of messages and rebuild the index
off this to ensure consistency. (Why does it need to do this
validation? Because in the event
This seems like a bug, no? It should just initiate the request not wait for
it to be written, there is no way for the user to reason about the state of
the send buffer.
-jay
On Monday, March 14, 2016, Jason Gustafson wrote:
> Hey Alexey,
>
> Asynchronous commit handling
..@timeli.io
>
>
>
> > On Mar 11, 2016, at 7:21 AM, Dick Davies <d...@hellooperator.net> wrote:
> >
> > Nice - I've read topics on the idea of a database as the 'now' view of a
> stream
> > of updates, it's a very powerful concept.
> >
> > Re
with the reactive-stream protocol for
> interoperability with libraries akka-stream and RxJava?
>
> Thanks,
> DaVe.
>
> David Buschman
> d...@timeli.io
>
>
>
> > On Mar 10, 2016, at 2:26 PM, Jay Kreps <j...@confluent.io> wrote:
> >
> > Hey a
Hey all,
Lot's of people have probably seen the ongoing work on Kafka Streams
happening. There is no real way to design a system like this in a vacuum,
so we put up a blog, some snapshot docs, and something you can download and
use easily to get feedback:
Hey Gligor,
Sorry for the rough edges. I think there are a variety of rough edges in
error messages here we can improve:
1. "Error ILLEGAL_GENERATION occurred while committing offsets for group
MetadataConsumerSpout" is obviously NOT the most intuitive error message,
it doesn't really
Hey Andrew,
Yeah I think the current state is that we did several design and prototypes
(both the transaction work and the idempotence design and the conditional
write KIP), but none of these offshoots is really fully rationalized with
the other ones. Slow progress in this area has been mainly
t 4:26 PM, Jay Kreps <j...@confluent.io> wrote:
> > The default semantics of the new consumer with auto commit are
> > at-least-once-delivery. Basically during the poll() call the commit will
> be
> > triggered and will commit the offset for the messages consum
The default semantics of the new consumer with auto commit are
at-least-once-delivery. Basically during the poll() call the commit will be
triggered and will commit the offset for the messages consumed during the
previous poll call. This is an advantage over the older scala consumer
where the
Check out Kafka Connect:
http://www.confluent.io/blog/how-to-build-a-scalable-etl-pipeline-with-kafka-connect
-Jay
On Wed, Feb 10, 2016 at 5:09 PM, R P wrote:
> Hello All,
> New Kafka user here. What is the best way to write Kafka data into HDFS?
> I have looked into
ou could track only offsets in memory and read back full values as
> needed to apply deltas, but this of course requires random reads into your
> Kafka topic (but may perform fine in practice depending on the workload).
>
> -Ewen
>
> On Fri, Jan 29, 2016 at 9:12 AM, Jay Kreps <
Also, most database provide a "full logging" option that let's you capture
the whole row in the log (I know Oracle and MySQL have this) but it sounds
like Mongo doesn't yet. That would be the ideal solution.
-Jay
On Fri, Jan 29, 2016 at 9:38 AM, Jay Kreps <j...@confluent.io> wrot
Hey Ewen, how come you need to get it all in memory for approach (1)? I
guess the obvious thing to do would just be to query for the record
after-image when you get the diff--e.g. just read a batch of changes and
multi-get the final values. I don't know how bad the overhead of this would
If you use the perf test without any bound on throughput it will
always try to send data faster than it can go out and build up a queue
of unsent data. So e.g. if your buffer is 1MB each send will be
blocked on waiting for the full 1MB of queued data to clear out and
get sent. This makes sense if
Hey guys,
Those docs were for a draft version of the API and I think they may be
a bit out of date. I'd stick with the javadocs linked here:
http://kafka.apache.org/documentation.html#newconsumerapi
-Jay
On Sun, Dec 6, 2015 at 7:40 AM, Dhyan Muralidharan
wrote:
>
Kafka should be robust against abrupt termination.
-Jay
On Sat, Dec 5, 2015 at 11:05 AM, Per Jahn wrote:
> Hi
>
> Are files that kafka writes to the file system robust against abrupt
> termination?
> Including fs meta data.
> And assuming it's used with a journaling fs.
>
It's worth noting that both the old and new consumer are identical in the
number of records fetched at once and this is bounded by the fetch size and
the number of partitions you subscribe to. The old consumer held these in
memory internally and waited for you to ask for them, the new consumer
Hey Everyone,
As you may have heard, Confluent is hosting the first ever Kafka
Summit. It'll be in San Francisco on Tuesday, April 26, 2016.
We'll be announcing open registration tomorrow, but I wanted to let
everyone here know first, and also let you know there is a $50
community discount. To
I think the point is that we should ideally try to cover all these in the
"upgrade" notes.
-Jay
On Tue, Dec 1, 2015 at 10:37 AM, Aditya Auradkar
wrote:
> Rajiv,
>
> By default, the quota is unlimited until you decide to configure them
> explicitly.
> And yes, we did get
Actually this may be something we can improve in the documentation. Calling
poll(1000) doesn't mean "check for new messages every second" but rather
"return the instant there are new messages, but if no messages come return
after a one second timeout passes".
So in that respect both the old and
Hey Dave,
We're separating the problem of getting data in and out of Kafka from the
problem of transforming it. If you think about ETL (Extract, Transform,
Load), what Kafka Connect does is E and L really really well and not T at
all; the focus in stream processing systems is T with E and L being
tions (each on
> different spindle) would lead to better read/write performance because
> of the aggregate bandwidth derived from parallel operations. But what
> I am seeing here defies my understanding. I just wonder if I did the
> benchmark tests wrong or I had the concept wrong.
The second command you give actually doesn't seem to double the memory
(maybe just a typo?). I can't explain why doubling buffer memory would
decrease throughput. The only effect of adding memory would be if you run
out, and then running out of memory would cause you to block and hence
lower
2 partitions on 2 brokers (on different physical server too) gave me the
> reading of 25MB/sec
>
> I just wanna know how to interpret these numbers so I can draw a pattern
> but so far this is
> not very consistent (more partitions = less throughput?)
>
> Cheers,
>
> Paul
Yuheng,
From the command you gave it looks like you are configuring the perf test
to send data as fast as possible (the -1 for target throughput). This means
it will always queue up a bunch of unsent data until the buffer is
exhausted and then block. The larger the buffer, the bigger the queue.
Hey James,
You are right the intended use of that was to have a way to capture some
very small metadata about your state at the time of offset commit in an
atomic way.
That field isn't exposed but we do need to add it to the new consumer api
(I think just no one has done it yet.
-Jay
On Mon,
It seems less weird if you think of the offset as the position of the
consumer, i.e. it is on record 5. In some sense the consumer is actually
in between records, i.e. if it has processed 4 and not processed 5 do you
think about your position as being on 4 or on 5? Well not on 4 because it
already
Hey Sivananda,
That's actually no longer true and likely a documentation bug. Where did
you see that?
-Jay
On Fri, Jul 17, 2015 at 9:35 AM, Sivananda Reddy sivananda2...@gmail.com
wrote:
Hi,
Kafka document ion says that the new producer is in Beta state, how safe is
it to use the new
This is almost certainly a client bug. Kafka's request format is size
delimited messages in the form
4 byte size NN byte payload
If the client sends a request with an invalid size or sends a partial
request the server will see effectively random bytes from the next request
as the size of the
If I recall correctly, setting log.retention.ms and log.retention.bytes to
-1 disables both.
On Fri, Jul 10, 2015 at 1:55 PM, Daniel Schierbeck
daniel.schierb...@gmail.com wrote:
On 10. jul. 2015, at 15.16, Shayne S shaynest...@gmail.com wrote:
There are two ways you can configure your
I have been thinking a little about this. I don't think CAS actually
requires any particular broker support. Rather the two writers just write
messages with some deterministic check-and-set criteria and all the
replicas read from the log and check this criteria before applying the
write. This
provide that will help you
understand what could be the issue?
Thanks,
Anuja
On Fri, Jun 5, 2015 at 2:36 PM, Jay Kreps jay.kr...@gmail.commailto:
jay.kr...@gmail.com wrote:
This sounds a lot like a bug we fixed in 0.8.2.0, no chance you are
running
that pre-release version
This sounds a lot like a bug we fixed in 0.8.2.0, no chance you are running
that pre-release version is there?
-Jay
On Wed, Jun 3, 2015 at 9:43 PM, Anuja Pundlik (apundlik) apund...@cisco.com
wrote:
Hi,
I am using Kafka 0.8.2.1.
We have 1 zookeeper, 3 kafka brokers.
We have 9 topics, out
Hey Ben,
The consumer actually doesn't promise to return records on any given poll()
call and even in trunk it won't return records on the first call likely.
Internally the reason is that it basically does one or two rounds of
non-blocking actions and then returns. This could include things like
I think java.lang.OutOfMemoryError: Map failed has usually been out of
address space for mmap if memory serves.
If you sum the length of all .index files while the service is running (not
after stopped), do they sum to something really close to 2GB? If so it is
likely either that the OS/arch is
Hey Garry,
Super interesting. We honestly never did a ton of performance tuning on the
producer. I checked the profiles early on in development and we fixed a few
issues that popped up in deployment, but I don't think anyone has done a
really scientific look. If you (or anyone else) want to dive
Does block.on.buffer.full=false do what you want?
-Jay
On Tue, May 5, 2015 at 1:59 AM, mete efk...@gmail.com wrote:
Hello Folks,
I was looking through the kafka.producer metrics on the JMX interface, to
find a good indicator when to trip the circuit. So far it seems like the
- which apparently everyone really liked,
and I don't think it is overly weird. It is very limited, but anyone who
needs more control over his sends already have plenty of options.
Thoughts?
Gwen
On Tue, Apr 28, 2015 at 5:29 PM, Jay Kreps jay.kr...@gmail.com wrote:
Hey guys
Hey guys,
The locking argument is correct for very small records ( 50 bytes),
batching will help here because for small records locking becomes the big
bottleneck. I think these use cases are rare but not unreasonable.
Overall I'd emphasize that the new producer is way faster at virtually all
something.
Thanks.
Jiangjie (Becket) Qin
On 4/24/15, 3:23 PM, Jay Kreps jay.kr...@gmail.com wrote:
That should work. I recommend using the performance tool cited in the blog
linked from the performance page of the website. That tool is more
accurate and uses the new producer.
On Fri, Apr 24
and write a
custom partitioner that uses hashcode().
Gwen
On Sun, Apr 26, 2015 at 7:57 AM, Jay Kreps jay.kr...@gmail.com wrote:
This was actually intentional.
The problem with relying on hashCode is that
(1) it is often a very bad hash function,
(2) it is not guaranteed
This was actually intentional.
The problem with relying on hashCode is that
(1) it is often a very bad hash function,
(2) it is not guaranteed to be consistent from run to run (i.e. if you
restart the jvm the value of hashing the same key can change!),
(3) it is not available outside the jvm so
tool against a 0.8.1 broker ?
-roshan
On 4/24/15 1:19 PM, Jay Kreps jay.kr...@gmail.com wrote:
Do make sure if you are at all performance sensitive you are using the new
producer api we released in 0.8.2.
-Jay
On Fri, Apr 24, 2015 at 12:46 PM, Roshan Naik ros...@hortonworks.com
wrote
If you are talking about within a single process, having one producer is
generally the fastest because batching dramatically reduces the number of
requests (esp using the new java producer).
-Jay
On Fri, Apr 24, 2015 at 4:54 AM, Manikumar Reddy manikumar.re...@gmail.com
wrote:
We have a 2 node
, Jay Kreps jay.kr...@gmail.com wrote:
If you are talking about within a single process, having one producer
is
generally the fastest because batching dramatically reduces the
number of
requests (esp using the new java producer).
-Jay
On Fri, Apr 24, 2015 at 4:54 AM, Manikumar
Hey Harsha,
A few comments:
Can you finish up the KIP there are some unfinished sentences and odd
whitespace things going on.
Here are the questions I think we should consider:
1. Do we need this at all given that we have the partition argument in
ProducerRecord which gives full control? I
On April 23, 2015 at 9:11:33 AM, Jay Kreps (jay.kr...@gmail.com) wrote:
Hey Harsha,
A few comments:
Can you finish up the KIP there are some unfinished sentences and odd
whitespace things going on.
Here are the questions I think we should consider:
1. Do we need this at all given
Hey Sriharsha,
Thanks for the excellent write-up.
Couple of minor questions:
1. Isn't the blocking handshake going to be a performance concern? Can we
do the handshake non-blocking instead? If anything that causes connections
to drop can incur blocking network roundtrips won't that eat up all
I think the blog post was giving that as an upper bound not a recommended
size. I think that blog goes through some of the trade offs of having more
or fewer partitions.
-Jay
On Tue, Apr 7, 2015 at 10:13 AM, François Méthot fmetho...@gmail.com
wrote:
Hi,
We initially had configured our
Are you using the 0.8.2 release or trunk?
-Jay
On Mon, Mar 30, 2015 at 1:35 AM, Sandeep Bishnoi
sandeepbishnoi.b...@gmail.com wrote:
Hi,
I have configured a kafka consumer as follows:
Properties props = new Properties();
// ..
// Populated properties
KafkaConsumer
:-)
On Thursday, March 19, 2015, James Cheng jch...@tivo.com wrote:
Those are pretty much the best javadocs I've ever seen. :)
Nice job, Kafka team.
-James
On Mar 19, 2015, at 9:40 PM, Jay Kreps jay.kr...@gmail.com
javascript:; wrote:
Err, here:
http://kafka.apache.org/083
The current work in progress is documented here:
On Thu, Mar 19, 2015 at 7:18 PM, Rajiv Kurian ra...@signalfuse.com wrote:
Is there a link to the proposed new consumer non-blocking API?
Thanks,
Rajiv
materials or links about it?
Thanks,
Xiao Li
On Mar 7, 2015, at 9:33 AM, Jay Kreps jay.kr...@gmail.com wrote:
Xiao,
FileChannel.force is fsync on unix.
To force fsync on every message:
log.flush.interval.messages=1
You are looking at the time based fsync, which, naturally, as you
Hey guys,
If we checked in obviously broken code on trunk, let's fix it now or revert
that change.
-Jay
On Sat, Mar 7, 2015 at 12:48 AM, Jiangjie Qin j...@linkedin.com.invalid
wrote:
Hi Tao,
Thanks a lot for finding the bug. We are actually rewriting the mirror
maker in KAFKA-1997 with a
= flushCheckpointMs,
TimeUnit.MILLISECONDS)
This thread is only time-controlled. It does not check the number of
messages.
Thank you,
Xiao Li
On Mar 5, 2015, at 11:59 AM, Jay Kreps jay.kr...@gmail.com wrote:
Hey Xiao,
That's not quite right. Fsync is controlled
I think this is great. I assume the form this would take would be a library
that implements the JMS api that wraps the existing java producer and
consumer?
Our past experience has been that trying to maintain all this stuff
centrally is too hard and tends to stifle rather than support innovation.
if this requirement is highly desirable to the others too?
Night,
Xiao Li
On Mar 4, 2015, at 9:00 AM, Jay Kreps jay.kr...@gmail.com wrote:
Hey Xiao,
Yeah I agree that without fsync you will not get durability in the case
of
a power outage or other correlated failure, and likewise without
This patent documents how it work. It is easy to understand, however,
you also need to consider the hash collision issues. This has been
implemented in IBM Q Replication since 2001.
Thanks,
Xiao Li
On Mar 3, 2015, at 3:36 PM, Jay Kreps jay.kr...@gmail.com wrote
, 2015, at 3:36 PM, Jay Kreps jay.kr...@gmail.com wrote:
Hey Josh,
As you say, ordering is per partition. Technically it is generally
possible
to publish all changes to a database to a single partition--generally
the
kafka partition should be high throughput enough to keep up. However
. That
will be my next focus. I am not sure if that can resolve our above concern.
BTW, do you have any plan to support mainframe?
Thanks,
Xiao Li
On Mar 4, 2015, at 8:01 AM, Jay Kreps jay.kr...@gmail.com wrote:
Hey Xiao,
1. Nothing prevents applying transactions transactionally
Broker replication is available now and fully documented in the docs. This
approach to availability has a lot of advantages discussed in that ticket
and the one below. Personally, having tried both approaches, I think this
is what most people should do (running a small highly available cluster
Hey Josh,
As you say, ordering is per partition. Technically it is generally possible
to publish all changes to a database to a single partition--generally the
kafka partition should be high throughput enough to keep up. However there
are a couple of downsides to this:
1. Consumer parallelism is
FWIW, this intensely confusing behavior is fixed in the new producer which
should give the expected result by default.
-Jay
On Mon, Mar 2, 2015 at 6:36 PM, Yang tedd...@gmail.com wrote:
Thanks. This is indeed the reason.
On Mar 2, 2015 4:38 PM, Christian Csar christ...@csar.us wrote:
I
They are mutually exclusive. Can you expand on the motivation/use for
combining them?
-Jay
On Sunday, March 1, 2015, Ivan Balashov ibalas...@gmail.com wrote:
Hi,
Do I understand correctly that compaction and deletion are currently
mutually exclusive?
Is it possible to compact recent
It is totally reasonable to have unlimited retention. We don't have an
explicit setting for this but you can set the time based retention policy
to something large
log.retention.hours=2147483647
which will retain the log for 245,146 years. :-)
-Jay
On Fri, Feb 27, 2015 at 4:12 PM, Warren Kiser
encryption and how the conversations between Kafka instances
are protected.
Christian
On Wed, Feb 25, 2015 at 11:51 AM, Jay Kreps j...@confluent.io wrote:
Hey guys,
One thing we tried to do along with the product release was start to put
together a practical guide for using Kafka. I wrote
+1
I think something like Kafka on AWS at Netflix would be hugely
interesting to a lot of people.
-Jay
On Mon, Feb 23, 2015 at 3:02 PM, Allen Wang aw...@netflix.com.invalid
wrote:
We (Steven Wu and Allen Wang) can talk about Kafka use cases and operations
in Netflix. Specifically, we can
with the interface we would end up with.
- Currently Cluster is not a public class so we'll have to think about
whether we want to make that public.
-Jay
On Sun, Feb 22, 2015 at 4:44 AM, Daniel Wegener
daniel.wege...@holisticon.de wrote:
Jay Kreps jay.kreps@... writes:
Hey Daniel,
partitionsFor
, these would be some
additional milliseconds to respond faster if we could spare
de/recompression.
Those are my thoughts about server side de/recompression. It would be
great if I could get some responses and thoughts back.
Jan
On 07.11.2014 00:23, Jay Kreps wrote:
I suspect it is possible
side by 60%. We plan to make it our
default partitioner.
On Sun, Feb 22, 2015 at 10:28 AM, Jay Kreps jay.kr...@gmail.com
javascript:; wrote:
Hey Daniel,
Yeah I think that would be doable. If you want to pursue it you would
need
to do a quick KIP just to get everyone on the same page
1 - 100 of 382 matches
Mail list logo