They are mutually exclusive. Can you expand on the motivation/use for
combining them?
-Jay
On Sunday, March 1, 2015, Ivan Balashov wrote:
> Hi,
>
> Do I understand correctly that compaction and deletion are currently
> mutually exclusive?
>
> Is it possible to compact recent segments and delete
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 wrote:
> Thanks. This is indeed the reason.
> On Mar 2, 2015 4:38 PM, "Christian Csar" wrote:
>
> > I believe you are seeing the beha
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
well
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 l
t; > Best wishes,
> > >
> > > Xiao Li
> > >
> > >
> > > On Mar 3, 2015, at 3:45 PM, Xiao wrote:
> > >
> > >> Hey Josh,
> > >>
> > >> Transactions can be applied in parallel in the consumer side based on
> >
nderstand, 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 wrote:
> >
> >> Hey J
; the recovery-point file. Any other ideas?
>
> I have not read the source codes for synchronous disk replication. 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,
>
&g
sync.
>
> Not sure if this requirement is highly desirable to the others too?
>
> Night,
>
> Xiao Li
>
> On Mar 4, 2015, at 9:00 AM, Jay Kreps wrote:
>
> > Hey Xiao,
> >
> > Yeah I agree that without fsync you will not get durability in the case
> of
> >
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.
ay = InitialTaskDelayMs,
> period = 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:5
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
wrote:
> Hi Tao,
>
> Thanks a lot for finding the bug. We are actually rewriting the mirror
> maker in KAFKA-1997 with a much simplified solutio
This is really cool. One minor thing is that there is a typo in the title.
I also think it would be good to give people a two sentence motivation of
the problem you want to solve up front so they can think about that as they
read through the article.
-Jay
On Tue, Mar 10, 2015 at 6:22 PM, Pierre-Y
; I believe transactional messaging is a critical feature. The design
> document is not very clear. Do you have more materials or links about it?
>
> Thanks,
>
> Xiao Li
>
>
> On Mar 7, 2015, at 9:33 AM, Jay Kreps wrote:
>
> > Xiao,
> >
> > FileChannel.fo
Err, here:
http://kafka.apache.org/083/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html
-Jay
On Thu, Mar 19, 2015 at 9:40 PM, Jay Kreps wrote:
> The current work in progress is documented here:
>
>
> On Thu, Mar 19, 2015 at 7:18 PM, Rajiv Kurian
> wrote:
The current work in progress is documented here:
On Thu, Mar 19, 2015 at 7:18 PM, Rajiv Kurian wrote:
> Is there a link to the proposed new consumer non-blocking API?
>
> Thanks,
> Rajiv
>
:-)
On Thursday, March 19, 2015, James Cheng 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 > wrote:
> >
> > Err, here:
> >
> http://kaf
single thread to get data from kafka, process them as well
> as run periodic jobs. For the periodic jobs to run we need a guarantee on
> how much time the poll call can take at most.
>
> Thanks!
>
> On Fri, Mar 20, 2015 at 6:59 AM, Rajiv Kurian
> wrote:
>
> > Thanks!
>
; > non-allocating methods can co-exist with the allocating ones and only
> users
> > who absolutely need to use them can make the trade-off of better
> > efficiency/performance for a slightly more error-prone and ugly API.
> >
> > Thoughts?
> >
> > Tha
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
> Kafka
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
wrote:
> Hi,
>
> We initially had configured our topics to have betwe
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 th
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 think
as another
> param this may or may not be used. Essentially its up to the Partitioner to
> disclose on what available information they are going to partition against.
>
> Thanks,
> Harsha
>
>
> On April 23, 2015 at 9:11:33 AM, 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 Reddy
wrote:
> We have a 2 node cluster with 100 topics.
The producer will attempt to batch records together into
> >>fewer
> >> requests whenever multiple records are being sent to the same
> >>partition."
> >>
> >> If I understand correctly, batching happens at topic/partition level,
> >
ainst a 0.8.1 broker ?
> -roshan
>
>
> On 4/24/15 1:19 PM, "Jay Kreps" 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, 20
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 no
read becomes a
> bottleneck, it is time to have more producers.
>
> Please correct me if I miss something.
>
> Thanks.
>
> Jiangjie (Becket) Qin
>
> On 4/24/15, 3:23 PM, "Jay Kreps" wrote:
>
> >That should work. I recommend using the performance tool cited
t; Perhaps we can say "wait for Sriharsha's partitioner patch and write a
> custom partitioner that uses hashcode()".
>
> Gwen
>
>
>
> On Sun, Apr 26, 2015 at 7:57 AM, Jay Kreps wrote:
>
> > This was actually intentional.
> >
> > The
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
us
The offset now begins at 0 and increases sequentially for each partition.
The offset is identical across all replicas of that partition on different
brokers, but amongst different partitions the offsets are independent (as
before). The offset of a committed message is unique within that
topic/parti
I think what Subhash is asking for is just plain old fashioned javadoc.
People are used to having that for java apis. We should do that, but I
think it would require moving the java-friendly wrappers into java. This
would probably be a good thing to do anyway just to get rid of all the
crazy scala
There is no reason the ClientUtils.fetchTopicMetadata should take broker
instances, that is totally unusable. I have a patch for this included in
KAFKA-642.
-Jay
On Mon, Dec 3, 2012 at 7:53 AM, Chris Curtin wrote:
> Hi,
>
> I was able to implement my own lookup code but have a few concerns abo
Hi Ben,
If I understand your question, you are asking if it is possible to decouple
or pipeline the sending of the request and the reading of the response.
E.g. do something like
send(req1)
send(req2)
recv(resp1)
recv(resp2)
I think your question is, if I do this how do I know what respons
, Dec 6, 2012 at 9:20 AM, Jay Kreps wrote:
> Hi Ben,
>
> If I understand your question, you are asking if it is possible to
> decouple or pipeline the sending of the request and the reading of the
> response. E.g. do something like
> send(req1)
> send(req2)
> recv(re
Cool, so did that make sense? tldr: requests are processed in the order
sent and responses are returned in the order of requests.
-Jay
On Thu, Dec 6, 2012 at 9:40 AM, ben fleis wrote:
> Ok. This is actually what I did in my first version of the code (the queue
> of outstanding requests). I m
ResponseVersion of Y? How would the
> cliient know how to parse ResponseVersion of Y. This makes me think that I
> don't understand the motivation behind ApiVersions. Can someone shed some
> light?
>
> Thanks
> Milind
>
>
>
>
>
>
>
> On Thu, Dec 6, 2
Okay I have fixed all the problems people found that I am aware of:
1. Correlation id used throughout
2. Brokers in metadata api not repeated per-partition
3. Removed creator id from broker because it is silly
4. All versions baselined to 0 (both ApiVersions and the message "magic"
version)
5. Remo
WRT the version I don't think it will really help to catch bugs. It is
definitely possible for us to have protocol bugs, but I don't think that
that version would catch the problem since the person breaking the protocol
would probably not change the version number. I also think it is confusing
for
I agree that BNFs are not the most readable thing in the world, but the
fact is that our protocol has variable length fields and lots of repeated
structures which are pretty confusing in a positional form. In practice I
think the code one writes looks a lot like the BNF (e.g.
readTopicMetadataReque
We would be interested in doing a similar thing--basically implementing ssl
for socket connections. These might be on a special SSL port, and would
begin with the SSL handshake. We haven't done any real research on how this
might work.
Be aware, also, that without zk being secure there are other p
Yes this is how it works. We do not log out to disk on the web service
machines, rather we use the async setting in the kafka producer from the
app and it directly sends all tracking and monitoring data to the kafka
cluster.
On Sun, Dec 9, 2012 at 12:47 PM, S Ahmed wrote:
> I was reading (or wa
instead of sending 2 line
> items.
>
> The key is to also figure out what size or time to queue before pushing
> them to kafka. For something like a page view, and other request
> information like browser, timestamp, querystring values, you could probably
> store a few hundred?
>
Hey Otis,
Not sure if you mean write up the protocol or write up more detailed
release notes and deployment steps. The protocol is documented here:
https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol
We will write up migration steps as we get closer to a stable release
All of the unit tests start and stop all their dependencies. You shouldn't
have to do anything prior to running ./sbt test. Did the test fail or did
it just print an exception? The sbt test runner will print exceptions and
other logging to the console. Some of the tests specifically invoke error
co
1. There are two kinds of limits: per server and overall. The per server
limits come from the fact that we use one directory and at least one file
per partition-replica. The normal rules of unix filesystem scalability
apply. The per server limits can be mitigated by adding more servers. The
overall
I put in was:
RedirectMatch 301 (.*) http://kafka.apache.org${1}
Seem right?
-Jay
On Fri, Dec 14, 2012 at 1:30 PM, Jay Kreps wrote:
> http://kafka.apache.org/
>
> I will set up redirects from the old to the new.
>
> If folks could sanity check the links, email addresses, eng
Hey Matan,
At LinkedIn we end up having lots of topics--some are very high volume and
some very very low volume. It works fine for low-volume topics.
At this time we don't support any kind of per-consumer throttling, that
would be a nice thing to have.
-Jay
On Sun, Dec 16, 2012 at 2:23 AM, Mat
presentations are no longer accessible
on the wiki, maybe there is another copy somewhere on the web or maybe you
have a copy you could upload?
-Jay
On Fri, Dec 14, 2012 at 4:40 PM, David Arthur wrote:
> On 12/14/12 4:30 PM, Jay Kreps wrote:
>
>> http://kafka.apache.org/
>>
>>
Any objections to my deleting the old github kafka repository. We kept it
around for posterity and to preserve the version control history. But I
found that it confuses people as to where we are hosted:
https://github.com/kafka-dev/kafka
We had an active replica of our apache git/svn replica, an
This is great! I added a link to mailing list page and we can integrate it
more fully as we improve the site.
-Jay
On Tue, Dec 18, 2012 at 10:56 AM, Otis Gospodnetic <
otis.gospodne...@gmail.com> wrote:
> Hi,
>
> Some time ago people on this list asked if we could add Kafka to
> search-hadoop.c
Hey Oliver,
That would be great! I will start a separate thread on this.
-Jay
On Sun, Dec 16, 2012 at 2:23 PM, Olivier Pomel wrote:
> > 7. There hasn't really been much a discussion on Logo, though we
> definitely
> > need one. I offered to act as "personal programming slave" to any of the
>
main is streams which could be visually kind of twisted and whispy.
-Jay
On Wed, Dec 19, 2012 at 10:03 AM, Jay Kreps wrote:
> A few people have asked about getting a logo for Kafka. I think this would
> be a great idea!
>
> Oliver has kindly offered to have a someone at Datadog take
r other work)? I ask,
> because knowing how the name "Kafka" relates to a distribute pub/sub system
> might help inspire ideas for a logo.
>
> -David
>
>
> On 12/19/12 1:03 PM, Jay Kreps wrote:
>
>> A few people have asked about getting a logo for Kafka.
I think the problem you are describing is that if a single broker is slow
all producers will come to a halt (because they all talk to this broker).
We don't have a great solution for this at the moment.
In our own usage for the first tier of data collection each producer
connects to a single brok
Hey All,
There has been interesting in getting something a little more sophisticated
then the Input- and OutputFormat we include in contrib for reading Kafka
data into HDFS.
Internally at LinkedIn we have had a pretty sophisticated system that we
use for Kafka ETL. It automatically discovers topi
This is a good point. We have discussed this a little bit before. The key
constraint is that with replication factor 1 you can choose one of the
following: (1) high availability, (2) correct semantic partitioning. That
is to say, if a particular partition is unavailable you have no choice but
to gi
setting in the consumer/producer that say "I don't care about
> > partitioning, just produce/consume wherever you can"
> >
> > This is probably not simple to implement, I'll add these ideas in the
> JIRA
> > and will pursue the discussion there.
> >
> > Maxime
> >
> > On Wed, Jan 9, 2013 at 5:18 PM, Jay Kreps wrote:
> >
> > > As a work around in the meantime you can probably run with
> > > replication--although it sounds like you don't really need it, it
> > shouldn't
> > > hurt.
> > >
> >
>
Yup, runs nightly with about 100 test scenarios. Code is in the main repo
under system_test/.
-Jay
On Tue, Jan 8, 2013 at 6:47 PM, S Ahmed wrote:
> I was just reading this:
> https://cwiki.apache.org/confluence/display/KAFKA/Performance+testing
>
> Sounds like a solid idea! Has this been impl
I have been noticing a lot of cool Kafka integrations floating around. I
took some time and went through github and emails and tried to update the
some of the pointers we have. S
Powered By - If you are using Kafka, add yourself here.
https://cwiki.apache.org/confluence/display/KAFKA/Powered
before others come up. That's what you get for producing awesome software
> companies rely on :-)
>
> Cheers,
> Olivier @ Datadog | http://datadog.com
>
>
>
> On Thu, Jan 10, 2013 at 12:22 PM, Evan Chan wrote:
>
> > I added a comment to the Powered By wiki. By
I can't speak for all users, but at LinkedIn we don't do this. We just run
Kafka as a high-availability system (i.e. something not allowed to be
down). These kind of systems require more care, but we already have a
number of such data systems. We chose this approach because local queuing
leads to d
Also, this wiki has a pretty good collection of presentations which may
give you ideas. If you want the source ppt or omnigraffle for any of the
presentations we made let us know.
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+papers+and+presentations
-Jay
On Wed, Jan 16, 2013 at 7:26
It's worth mentioning that we are interested in exploring potential
generalizations of the producer and consumer API, but as a practical matter
most of the committers are working on getting a stable 0.8 release out the
door. So an improved consumer and producer api would be a 0.9 feature.
If you h
ut, implement true hasNext functionality, for example)
>
> thanks,
> Evan
>
>
> On Mon, Jan 21, 2013 at 9:27 AM, Jay Kreps wrote:
>
> > It's worth mentioning that we are interested in exploring potential
> > generalizations of the producer and consumer API, b
Hi Evan,
Makes sense. Is your goal in separating the client shrinking the jar size?
or just general cleanliness?
-Jay
On Tue, Jan 22, 2013 at 10:53 AM, Evan Chan wrote:
> Jay,
>
> Comments inlined.
>
> On Tue, Jan 22, 2013 at 10:15 AM, Jay Kreps wrote:
>
> > Hey Ev
We put a lot of info here:
https://cwiki.apache.org/confluence/display/KAFKA/Operations
Does that help?
-Jay
On Tue, Jan 22, 2013 at 7:14 PM, S Ahmed wrote:
> In the wild, what sort of memory usage patterns have you guys seen with
> kafka?
>
> I'm not that well versed with java and its memory
There is a setting that controls the maximum message size. This is to
ensure the messages can be read on the server and by all consumers without
running out of memory or exceeding the consumer fetch size. In 0.7.x this
setting is controlled by the broker configuration max.message.size.
-Jay
On T
Ack, right you are Neha, my bad.
WRT to how to set the maximum there are two considerations:
1. It should be smaller then the fetch size your consumers use
2. Messages are fully instantiated in memory so obscenely large messages
(say hundreds of mb) will cause a lot of memory allocation churn/prob
This is super cool. Are we linking the right repo on the ecosystem page?
https://cwiki.apache.org/confluence/display/KAFKA/Ecosystem
-Jay
On Thu, Jan 31, 2013 at 1:29 AM, Jibran Saithi wrote:
> Hey,
>
> I know this has come up a few times, so thought I'd share a bit of code
> we've been using
Cool, well definitely add it to that ecosystem page to help people find it.
-Jay
On Thu, Jan 31, 2013 at 9:38 AM, Jibran Saithi wrote:
> Yeah, we use supervisor which seems to work fine for us.
>
> On 31 January 2013 14:57, S Ahmed wrote:
>
> > Great thanks.
> >
> > BTW, that's not a daemon ser
The easiest way to diagnose is to enable GC logging on both the consumer
and the zk instance and see if you have long pauses.
-Jay
On Tue, Feb 5, 2013 at 5:46 PM, Neha Narkhede wrote:
> >> Unable to reconnect to ZooKeeper service, session 0x33c981ab95100ed
> has expired, closing socket connecti
For our long running testing at LinkedIn we actually just use the mirror
maker tool and fork off a copy of the full production streams and throw it
at the test cluster. This is, of course, only useful if you have a
high-load cluster to replay...
-Jay
On Wed, Feb 6, 2013 at 11:27 AM, S Ahmed wro
This is awesome. When you get it fully functional add it to the clients
page so people can find it.
-Jay
On Mon, Feb 11, 2013 at 11:40 PM, Bob Potter wrote:
> Howdy,
>
> I just pushed the initial version of a new Ruby client that implements the
> new 0.8 wire protocol and includes a producer w
new replication features?
>
> Thanks,
>
> Chris
>
>
> On Wed, Jan 16, 2013 at 12:35 PM, Jay Kreps wrote:
>
>> Also, this wiki has a pretty good collection of presentations which may
>> give you ideas. If you want the source ppt or omnigraffle for any of the
>&g
Yes, it will work. We do this at LinkedIn, actually.
-Jay
On Thu, Feb 14, 2013 at 12:15 PM, Jamie Wang wrote:
> I have a situation where we have constraint on the number of processes we can
> use and hence, is it possible to load kafka server within another Java
> process? Looking at the kafka
To be clear: to lose data in the filesystem you need to hard kill the
machine. A hard kill of the process will not cause that.
-Jay
On Tue, Feb 19, 2013 at 8:25 AM, Jun Rao wrote:
> Jason,
>
> Although messages are always written to the log segment file, they
> initially are only in the file sys
ould happen if there is machine crash? Then when we restart the
> server, we will see the 00.log file with only 100 messages in it?
>
> Thanks,
>
> Jason
>
> On Wed, Feb 20, 2013 at 1:59 AM, Jay Kreps wrote:
>> To be clear: to lose data in the filesystem you need to har
You can do this and it should work fine. You would have to keep adding
machines to get disk capacity, of course, since your data set would
only grow.
We will keep an open file descriptor per file, but I think that is
okay. Just set the segment size to 1GB, then with 10TB of storage that
is only 10
just schema, but schema + other contextual information common to
> all events from a producer (e.g. source machine, application, app version
> etc).
>
>
> On Feb 21, 2013, at 7:26 PM, Jay Kreps wrote:
>
>> You can do this and it should work fine. You would have to keep adding
>
We limit individual segment files to 2GB for slightly esoteric
reasons. There is no limit for the log as a whole so limiting the
segments to 2GB does not constrain how much data you can retain.
-Jay
On Mon, Mar 25, 2013 at 11:42 AM, S Ahmed wrote:
> Is there any limit to how large a log file can
You should be able to just bounce the broker. Our default policy is
that if we run out of space we shut down the broker automatically as
in that case there is no guarantee on what has been written to disk.
On startup if a clean shutdown hasn't been performed the broker should
run a recovery procedu
message, the broker should kill itself and then fix the log
on recovery when that occurs.
-Jay
On Sat, Apr 13, 2013 at 11:27 AM, Jay Kreps wrote:
> You should be able to just bounce the broker. Our default policy is
> that if we run out of space we shut down the broker automatically as
>
I think there is a misunderstanding here. The log file division is
completely transparent to the consumer. The consumer consumes by
offset (0,1,2,3,...) and these offsets span log files (i.e. there is a
single, monotonically increasing set of offsets over all log files in
a partition). In other wor
The intention is to allow the use of multiple disks without RAID or
logical volume management. We have found that there are a lot of
downsides to RAID--in particular a huge throughput hit. Since we
already have a parallelism model due to partitioning and a fault
tolerance model with replication RAI
ized storage,
> using some sort of weighting scheme, etc.).
>
> Jason
>
>
> On Thu, Jun 20, 2013 at 11:07 AM, Jay Kreps wrote:
>
>> The intention is to allow the use of multiple disks without RAID or
>> logical volume management. We have found that there are a lot
It's also worth mentioning why new slave machines need to truncate
back to a known good point.
When a new server joins the cluster and already has some data on disk
we cannot blindly trust its log as it may have messages that were
never committed (for example if it was the master and then crashed
I have noticed we don't do a good job of updating the FAQs. Would we
do better if I migrated it to the wiki so it was easier to edit?
-Jay
Yeah, that is true. I thought I documented that, but looking at the
protocol docs, it looks like I didn't.
I agree this is kind of a pain in the ass. It was an important
optimization in 0.7 because we didn't know where the message
boundaries were but in 0.8 we have a fast way to compute message
bo
I think InfoChimps is doing some support/consulting around Kafka.
-Jay
On Thu, Jun 27, 2013 at 10:30 AM, d0t wrote:
> Hello all,
>
> Are there any vendors who provide commercial support for Kafka?
Keylis wrote:
> Jay. I assume this is problem exists in the consumer. How this can this
> problem be triggered so I could test my high level consumer.
>
> Thanks
>
> On Jun 26, 2013, at 9:21 AM, Jay Kreps wrote:
>
>> Yeah, that is true. I thought I documented that, but look
Hey Folks,
I did a pass on the website. Changes:
1. Rewrote the 0.8 quickstart and included a section on running in
distributed mode.
2. Fixed up the styles a bit.
3. Fixed the bizarre menu thing with 0.7 and 0.8 specific docs.
4. Re-wrote the copy on the front page.
I would love to get any feedb
This is correct. It would be possible to change the server behavior to
avoid the smaller fetch responses but it would be more complex to
implement and there wouldn't be much benefit to this.
You can tell you are at the end of the log because either (1) you get
an empty message set back, or (2) the
bramanian"
>>>wrote:
>>>
>>>>Looks much better.
>>>>
>>>>1. We need to update FAQ for 0.8
>>>>2. We should probably have a separate section for implementation.
>>>>3. The migration tool explanation seems to be hard to
left
> panel. This doesn't leave enough room for adding future releases.
>
> Jun
>
>
>
> On Sat, Jun 29, 2013 at 3:09 PM, Jay Kreps wrote:
>
> > Ack, nice catch--that migration tool thing was due to bad html, I
> > forgot to close the link.
> >
> &g
ill update
> the page today and we can continue editing it. Sounds good?
>
> On 7/1/13 9:42 AM, "Jay Kreps" wrote:
>
> >Yeah thanks for the feedback, that's helpful. Here was my thinking:
> >1. I think it just makes sense to have one design and implementation
The recording for the user group talks is available here:
http://www.ustream.tv/linkedin-events
-Jay
On Wed, Jun 26, 2013 at 8:22 AM, Jun Rao wrote:
> Hi, Everyone,
>
> We have finalized our agenda of the meetup Thursday evening, with Speakers
> from LinkedIn, RichRelevance, Netflix and Square
Erp, actually I don't see a way to get the recording there, nm.
-Jay
On Tue, Jul 2, 2013 at 12:47 PM, Jay Kreps wrote:
> The recording for the user group talks is available here:
> http://www.ustream.tv/linkedin-events
>
> -Jay
>
>
> On Wed, Jun 26, 2013 at 8:22 A
This is exactly right. Partitions is configurable so set it to some
reasonable upper bound on the concurrency you desire. To give further
examples, let's say you have 5 threads:
- if you have 2 partitions only two threads will get data
- if you have 100 partitions each thread will get 20 partitions
We currently have a contrib package for consuming and producing messages
from mapreduce (
https://git-wip-us.apache.org/repos/asf?p=kafka.git;a=tree;f=contrib;h=e53e1fb34893e733b10ff27e79e6a1dcbb8d7ab0;hb=HEAD
).
We keep running into problems (e.g. KAFKA-946) that are basically due to
the fact tha
101 - 200 of 495 matches
Mail list logo