On Tue, May 3, 2016 at 1:00 PM Banias H wrote:
> I should add Flume is not an option for various reasons.
>
> On Tue, May 3, 2016 at 2:49 PM, Banias H wrote:
>
> > We use Kafka (0.9.x) internally in our pipeline and now we would like to
> > ingest application logs sitting in local file system of
I'm keen to hear about how to work one's way out of a filled partition
since I've run into this many times after having tuned retention bytes or
retention (time?) incorrectly. The proper path to resolving this isn't
obvious based on my many harried searches through documentation.
I often end up st
I too could benefit from an updated roadmap.
We're in a similar situation where some components in our stream processing
stack could use an overhaul, but I'm waiting for the offset API to be fully
realized before doing any meaningful planning.
On Fri, Aug 1, 2014 at 11:52 AM, Jonathan Weeks
wro
now) caused that broker to drop the partitions and leave the
replica set finally.
> Jun
>
>
>
> On Wed, May 21, 2014 at 10:09 PM, David Birdsong
> wrote:
>
> > On Wed, May 21, 2014 at 9:06 PM, Jun Rao wrote:
> >
> > > Expansion can be done by following
&
anks,
>
> Jun
>
>
> On Wed, May 21, 2014 at 3:23 PM, David Birdsong >wrote:
>
> > Any suggestions? I'm kind of in a bind in that I don't understand how to
> > grow the cluster when more capacity is needed--which happens to be right
> > now
ff the old ones
and forever retire their IDs freeing up my old hardware to come back online
as a new kafka broker ID.
On Wed, May 21, 2014 at 9:16 AM, David Birdsong wrote:
>
>
>
> On Wed, May 21, 2014 at 9:11 AM, David Birdsong
> wrote:
>
>> Here's the reassign
On Wed, May 21, 2014 at 9:11 AM, David Birdsong wrote:
> Here's the reassignment json and current "describe" output:
> https://gist.github.com/davidbirdsong/32cd0c4f49496a6a32e5
>
>
> In my re-assignment json, I tried to re-assign to 2 when the repl is set
f the old broker?
>
> Thanks,
>
> Jun
>
>
> On Wed, May 21, 2014 at 8:21 AM, David Birdsong >wrote:
>
> > I did that and so now the topic has 4 replicas for a repl count of 3, but
> > only the 'new' replicas exist in the ISR.
> >
> > The
Jun
>
>
> On Wed, May 21, 2014 at 1:23 AM, David Birdsong >wrote:
>
> > Is there a way to reduce the replication count? I'm trying to spread
> > existing partitions across more brokers, but it's hard to decomm a
> broker.
> > Reducing repl count woul
Is there a way to reduce the replication count? I'm trying to spread
existing partitions across more brokers, but it's hard to decomm a broker.
Reducing repl count would suffice for now.
Any tips?
I'm running a mix of 0.8.1.1 and 0.8.1 (I'm upgrading now.)
On Sun, Mar 9, 2014 at 6:09 PM, Balasubramanian Jayaraman (Contingent) <
balasubramanian.jayara...@autodesk.com> wrote:
> Thanks Martin. We are still in the design phase. I wanted to clarify my
> doubt on the relation between parallelism and partitions.
>
kafka is a distributed, ordered commit lo
t using the scala/jvm client libs. I'm on my
own in golang land and I've had to implement a high-level consumer on my
own which has been an adventure.
> Guozhang
>
>
> On Mon, Mar 3, 2014 at 4:16 PM, David Birdsong >wrote:
>
> > On Mon, Mar 3, 2014 at 4:00 PM, Guo
On Mon, Mar 3, 2014 at 4:00 PM, Guozhang Wang wrote:
> Hi Chris,
>
> In 0.9 we will have just one "broker list", i.e. the list of brokers read
> from the config file will be updated during bootstraping and all the future
> metadata refresh operations. This feature should lift this limit you are
>
Yeah, I've only skimmed this, but I think I might have something.
All non-vpc type ec2 nodes come with an external IP address and an internal
IP address. The external IP address is what grants the node access to the
internet--makes it publicly routable. The mechanism by which the external
IP addre
On Wed, Feb 5, 2014 at 2:22 PM, Robert Rodgers wrote:
> this would be great to add to the operational section of the Kafka
> documentation.
>
So is this a way to delete topics? Does this work?
>
> On Feb 5, 2014, at 2:18 PM, Andrew Otto wrote:
>
> >> - Increasing num.replica.fetchers (defaul
exit is the offset thread and it has a direct communication channel to
the consumer threads so it waits for those channels to be closed before
it's last flush and exit.
> Guozhang
>
>
Thanks for the input!
>
> On Mon, Jan 27, 2014 at 3:03 PM, David Birdsong >wrote:
>
Hey All, I've been cobbling together a high-level consumer for golang
building on top of Shopify's Sarama package and wanted to run the basic
design by the list and get some feedback or pointers on things I've missed
or will eventually encounter on my own.
I'm using zookeeper to coordinate topic-p
On Fri, Dec 13, 2013 at 11:21 AM, Neha Narkhede wrote:
> Partition movement is not an automatic operation in Kafka yet. You need to
> use the partition reassignment tool -
>
> https://cwiki.apache.org/confluence/display/KAFKA/Replication+tools#Replicationtools-6.ReassignPartitionsTool
> .
>
>
> Al
r topics since a partition is an SPOF.
> On Thu, Dec 12, 2013 at 9:32 PM, David Birdsong >wrote:
>
> > On Thu, Dec 12, 2013 at 9:28 PM, Jun Rao wrote:
> >
> > > Could you try starting from scratch again? The recent fix that we had
> may
> >
ere some state file? I have
other topics that aren't problematic. Is there no topic-level emergency
button to push?
> On Thu, Dec 12, 2013 at 8:45 PM, David Birdsong >wrote:
>
> > I was running a 2-node kafka cluster off github trunnk at:
> > eedbea6526986783257ad0e025c45
On Thu, Dec 12, 2013 at 9:28 PM, Guozhang Wang wrote:
> David,
>
> Could you try to see if this is due to
> https://issues.apache.org/jira/browse/KAFKA-1178?
>
> Guozhang
>
Which node do I look for this on? Leader? ISR-candidate? Controller?
>
>
> On Thu,
I was running a 2-node kafka cluster off github trunnk at:
eedbea6526986783257ad0e025c451a8ee3d9095
...for a few weeks with no issues. I recently downloaded the 0.8 stable
version, configured/started two new brokers with 0.8.
I successfully reassigned all but 1 partition from the older pair to th
22 matches
Mail list logo