Todd Palino created KAFKA-6559:
--
Summary: Iterate record sets before calling Log.append
Key: KAFKA-6559
URL: https://issues.apache.org/jira/browse/KAFKA-6559
Project: Kafka
Issue Type
; fed to the tool. I tried to search using the zk shell but couldn't find...
>
> Sagar.
>
> On Fri, Dec 22, 2017 at 7:45 PM, Todd Palino <tpal...@gmail.com> wrote:
>
> > Preferred replica election is naive. It will always follow the order of
> the
> > replicas
Preferred replica election is naive. It will always follow the order of the
replicas as they are set. So if you want to set the default leader, just
make it the first replica in the list for the partition. We build the JASON
this way all the time.
-Todd
On Dec 22, 2017 6:46 AM, "Sagar"
ssl
>
> Ismael
>
>
> On Tue, Sep 6, 2016 at 1:48 PM, Todd Palino <tpal...@gmail.com> wrote:
>
> > Yeah, that's why I mentioned it with a caveat :) Someone (I can't recall
> > who, but it was someone I consider reasonably knowledgable as I actually
> > gave i
[
https://issues.apache.org/jira/browse/KAFKA-5056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Todd Palino updated KAFKA-5056:
---
Summary: Shuffling of partitions in old consumer fetch requests removed
(was: Shuffling
[
https://issues.apache.org/jira/browse/KAFKA-5056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Todd Palino updated KAFKA-5056:
---
Reviewer: Joel Koshy
Status: Patch Available (was: In Progress)
> Shuffling of partiti
[
https://issues.apache.org/jira/browse/KAFKA-5056?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Work on KAFKA-5056 started by Todd Palino.
--
> Shuffling of partitions in old consume fetch requests remo
Todd Palino created KAFKA-5056:
--
Summary: Shuffling of partitions in old consume fetch requests
removed
Key: KAFKA-5056
URL: https://issues.apache.org/jira/browse/KAFKA-5056
Project: Kafka
ardware changes are less common
> than thread pool configuration changes.
>
> Thanks,
>
> Jun
>
> On Tue, Mar 7, 2017 at 4:45 PM, Todd Palino <tpal...@gmail.com> wrote:
>
> > I’ve been following this one on and off, and overall it sounds good to
>
I’ve been following this one on and off, and overall it sounds good to me.
- The SSL question is a good one. However, that type of overhead should be
proportional to the bytes rate, so I think that a bytes rate quota would
still be a suitable way to address it.
- I think it’s better to make the
[
https://issues.apache.org/jira/browse/KAFKA-1342?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15893728#comment-15893728
]
Todd Palino commented on KAFKA-1342:
[~wushujames], I'm not sure about increasing the number
t; > > 2011/12/16/setting-up-xfs-the-simple-edition/). This way, writing
> a
> > > > block
> > > > > doesn't require a read first. A large block size may increase the
> > > amount
> > > > of
> > > > > data writes, when the same block has to be written to disk multiple
> > > > times.
> > > > > However, this is probably ok in Kafka's use case since we batch the
> > I/O
> > > > > flush already. As you can see, we will be adding some complexity to
> > > > support
> > > > > JBOD in Kafka one way or another. If we can tune the performance of
> > > RAID5
> > > > > to match that of RAID10, perhaps using RAID5 is a simpler solution.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Fri, Feb 24, 2017 at 10:17 AM, Dong Lin <lindon...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hey Jun,
> > > > > >
> > > > > > I don't think we should allow failed replicas to be re-created on
> > the
> > > > > good
> > > > > > disks. Say there are 2 disks and each of them is 51% loaded. If
> any
> > > > disk
> > > > > > fail, and we allow replicas to be re-created on the other disks,
> > both
> > > > > disks
> > > > > > will fail. Alternatively we can disable replica creation if there
> > is
> > > > bad
> > > > > > disk on a broker. I personally think it is worth the additional
> > > > > complexity
> > > > > > in the broker to store created replicas in ZK so that we allow
> new
> > > > > replicas
> > > > > > to be created on the broker even when there is bad log directory.
> > > This
> > > > > > approach won't add complexity in the controller. But I am fine
> with
> > > > > > disabling replica creation when there is bad log directory that
> if
> > it
> > > > is
> > > > > > the only blocking issue for this KIP.
> > > > > >
> > > > > > Whether we store created flags is independent of whether/how we
> > store
> > > > > > offline replicas. Per our previous discussion, do you think it is
> > OK
> > > > not
> > > > > > store offline replicas in ZK and propagate the offline replicas
> > from
> > > > > broker
> > > > > > to controller via LeaderAndIsrRequest?
> > > > > >
> > > > > > Thanks,
> > > > > > Dong
> > > > > >
> > > > >
> > > >
> > >
> >
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
er we added headers, wouldn't you be in the
> business of making sure everyone uses them properly? Making sure
> everyone includes the right headers you need, not using the header
> names you intend to use, etc. I don't think the "policing" business
> will ever go away.
>
> On Thu,
do
> headers in Kafka. I absolutely don't mind it if you do it...
> I think the Apache convention for "good idea, but not willing to put
> any work toward it" is +0.5? anyway, that's what I was trying to
> convey :)
>
> On Thu, Dec 1, 2016 at 3:05 PM, Todd Palino <tpal...@gma
n or you (Todd)... I am
> just looking for something concrete we can do to move the discussion
> along to the yummy design details (which is the argument I really am
> looking forward to).
>
> On Thu, Dec 1, 2016 at 1:53 PM, Todd Palino <tpal...@gmail.com> wrote:
> > So
on a workable standard that we can adopt.
-Todd
On Thu, Dec 1, 2016 at 1:39 PM, Todd Palino <tpal...@gmail.com> wrote:
> C. per message encryption
>> One drawback of this approach is that this significantly reduce the
>> effectiveness of compression, which happens o
>
> C. per message encryption
> One drawback of this approach is that this significantly reduce the
> effectiveness of compression, which happens on a set of serialized
> messages. An alternative is to enable SSL for wire encryption and rely on
> the storage system (e.g. LUKS) for at rest
[
https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15700938#comment-15700938
]
Todd Palino commented on KAFKA-3959:
+1 as well. I think keeping the config/server.properties file
[
https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15668010#comment-15668010
]
Todd Palino commented on KAFKA-3959:
As noted, I just want the config enforced. If RF=3 is configured
layer.
-Todd
On Tuesday, September 6, 2016, Ismael Juma <ism...@juma.me.uk> wrote:
> Hi Todd,
>
> Thanks for sharing your experience enabling TLS in your clusters. Very
> helpful. One comment below.
>
> On Sun, Sep 4, 2016 at 6:28 PM, Todd Palino <tpal...@gmail.
SSL working with
> reasonably good performance, but the current performance isn't promising.
> Expecting this to be fixed in the next couple of months and have it
> available in 0.10.x is probably too much to expect, but if we know the
> plans around this, we should be able to come
n is now a member
> of
> >> > > Kafka
> >> > > > PMC.
> >> > > >
> >> > > > Congratulations, Gwen!
> >> > > >
> >> > > > Jun
> >> > > >
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > Grant Henke
> >> > Software Engineer | Cloudera
> >> > gr...@cloudera.com <javascript:;> | twitter.com/gchenke |
> >> > linkedin.com/in/granthenke
> >> >
> >>
> >>
> >> --
> >> Ashish h
> >>
> >
> >
> >
> > --
> > -- Guozhang
>
>
>
> --
> Gwen Shapira
> Product Manager | Confluent
> 650.450.2760 | @gwenshap
> Follow us: Twitter | blog
>
--
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
ith starting the vote on the current
> proposal?
>
> Jun
>
> On Thu, Aug 18, 2016 at 11:00 AM, Todd Palino <tpal...@gmail.com> wrote:
>
> > Joel just reminded me to take another look at this one :) So first off,
> > this is great. It’s something that we definit
t; > >>>>>>>>>>>>> On Tue, Aug 9, 2016 at 10:49 AM, Joel Koshy <
> >> > > >>>>>> jjkosh...@gmail.com <javascript:;>>
> >> > > >>>>>>>>>>> wrote:
> >> > >
he leader, and “move” partitions from the throttled
> > > >>>> replica
> > > >>>>>>>> fetchers
> > > >>>>>>>>>>>> to
> &g
[
https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425756#comment-15425756
]
Todd Palino commented on KAFKA-4050:
So first off, yes, the thread dump (which [~jjkoshy] posted
[
https://issues.apache.org/jira/browse/KAFKA-4050?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15423400#comment-15423400
]
Todd Palino commented on KAFKA-4050:
It appears to be called every time something needs
Todd Palino created KAFKA-4050:
--
Summary: Allow configuration of the PRNG used for SSL
Key: KAFKA-4050
URL: https://issues.apache.org/jira/browse/KAFKA-4050
Project: Kafka
Issue Type
[
https://issues.apache.org/jira/browse/KAFKA-3959?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15419365#comment-15419365
]
Todd Palino commented on KAFKA-3959:
Agree with Onur 100% here. We've been running into this a lot
[
https://issues.apache.org/jira/browse/KAFKA-3797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15323930#comment-15323930
]
Todd Palino commented on KAFKA-3797:
Obviously we can't do something like that with a running consumer
[
https://issues.apache.org/jira/browse/KAFKA-3797?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15318854#comment-15318854
]
Todd Palino commented on KAFKA-3797:
I think the first option is more reasonable, and provides
[
https://issues.apache.org/jira/browse/KAFKA-3725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15312288#comment-15312288
]
Todd Palino commented on KAFKA-3725:
I'll take a look at this and put together a PR with some updates
[
https://issues.apache.org/jira/browse/KAFKA-3725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Todd Palino reassigned KAFKA-3725:
--
Assignee: Todd Palino
> Update documentation with regards to
>
> I'm on 0.9.0.1. "broker-list" is invalid and zookeeper is required
> regardless of the bootstrap-server parameter.
>
>
> Thanks,
>
> Cliff
>
> On Sun, May 8, 2016 at 7:35 PM, Todd Palino <tpal...@gmail.com> wrote:
>
> > It looks li
and may contain confidential and privileged
> information. Any unauthorized use of this email is strictly prohibited.
> ©2016 Signal. All rights reserved.
>
--
*—-*
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
[
https://issues.apache.org/jira/browse/KAFKA-3174?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15126292#comment-15126292
]
Todd Palino commented on KAFKA-3174:
Yeah, definitely no problems with Java 1.8. We've been running
> > > > > > > topic deletion use-case, maybe it is better count them with
> > > onSend()
> > > > > and
> > > > > > > onAcknowledge() -- whether all messages sent were
> acknowledged. I
> > > > also
> > > > >
message validator then the audit can be done
> > > on-arrival
> > > > and we won't need a console auditor.
> > > >
> > > > One potential issue in this approach and any elaborate on-arrival
> > > > processing for that matter is that you may need t
t; following API to producer and consumer interceptors:
> > >
> > > ProducerInterceptor:
> > > SerializedKeyValue onEnqueued(TopicPartition tp, ProducerRecord<K, V>
> > > record, SerializedKeyValue serializedKeyValue);
> > >
> > > Consum
gt; > > >
> > > > > Thanks,
> > > > > Anna
> > > >
> > > >
> > > >
> > > >
> > > > This email and any attachments may contain confidential and
> privileged
> > > > material for the sole use of the intended recipient. Any review,
> > copying,
> > > > or distribution of this email (or any attachments) by others is
> > > prohibited.
> > > > If you are not the intended recipient, please contact the sender
> > > > immediately and permanently delete this email and any attachments. No
> > > > employee or agent of TiVo Inc. is authorized to conclude any binding
> > > > agreement on behalf of TiVo Inc. by email. Binding agreements with
> TiVo
> > > > Inc. may only be made by a signed written agreement.
> > > >
> > >
> > >
> > >
> > > --
> > > Thanks,
> > > Neha
> > >
> >
>
--
*—-*
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
[
https://issues.apache.org/jira/browse/KAFKA-3015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15065636#comment-15065636
]
Todd Palino commented on KAFKA-3015:
[~jkreps] So yes, I'm essentially saying that I would prefer
[
https://issues.apache.org/jira/browse/KAFKA-3015?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15065239#comment-15065239
]
Todd Palino commented on KAFKA-3015:
While this seems good on the surface, it makes it impossible
:
> > > >
> > > >> I would like to start a discussion around the work that has started
> in
> > > >> regards to KIP-30
> > > >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-30+-+Allow+for+brokers+to+have+plug-able+consensus+and+meta+data+storage+sub+systems
> > > >>
> > > >> The impetus for working on this came a lot from the community. For
> the
> > > last
> > > >> year(~+) it has been the most asked question at any talk I have
> given
> > > >> (personally speaking). It has come up a bit also on the mailing list
> > > >> talking about zkclient vs currator. A lot of folks want to use Kafka
> > but
> > > >> introducing dependencies are hard for the enterprise so the goals
> > behind
> > > >> this is making it so that using Kafka can be done as easy as
> possible
> > > for
> > > >> the operations teams to-do when they do. If they are already
> > supporting
> > > >> ZooKeeper they can keep doing that but if not they want (users) to
> use
> > > >> something else they are already supporting that can plug-in to-do
> the
> > > same
> > > >> things.
> > > >>
> > > >> For the core project I think we should leave in upstream what we
> have.
> > > This
> > > >> gives a great baseline regression for folks and makes the work for
> > > "making
> > > >> what we have plug-able work" a good defined task (carve out, layer
> in
> > > API
> > > >> impl, push back tests pass). From there then when folks want their
> > > >> implementation to be something besides ZooKeeper they can develop,
> > test
> > > and
> > > >> support that if they choose.
> > > >>
> > > >> We would like to suggest that we have the plugin interface be Java
> > based
> > > >> for minimizing depends for JVM impl. This could be in another
> > directory
> > > >> something TBD /.
> > > >>
> > > >> If you have a server you want to try to get it working but you
> aren't
> > on
> > > >> the JVM don't be afraid just think about a REST impl and if you can
> > work
> > > >> inside of that you have some light RPC layers (this was the first
> pass
> > > >> prototype we did to flush-out the public api presented on the KIP).
> > > >>
> > > >> There are a lot of parts to working on this and the more
> > > implementations we
> > > >> have the better we can flush out the public interface. I will leave
> > the
> > > >> technical details and design to JIRA tickets that are linked through
> > the
> > > >> confluence page as these decisions come about and code starts for
> > > reviews
> > > >> and we can target the specific modules having the context separate
> is
> > > >> helpful especially if multiple folks are working on it.
> > > >> https://issues.apache.org/jira/browse/KAFKA-2916
> > > >>
> > > >> Do other folks want to build implementations? Maybe we should start
> a
> > > >> confluence page for those or use an existing one and add to it so we
> > can
> > > >> coordinate some there to.
> > > >>
> > > >> Thanks!
> > > >>
> > > >> ~ Joe Stein
> > > >> - - - - - - - - - - - - - - - - - - -
> > > >> [image: Logo-Black.jpg]
> > > >> http://www.elodina.net
> > > >>http://www.stealth.ly
> > > >> - - - - - - - - - - - - - - - - - - -
> > > >>
> > >
> > >
> >
>
--
*—-*
*Todd Palino*
Staff Site Reliability Engineer
Data Infrastructure Streaming
linkedin.com/in/toddpalino
On Thu, Dec 3, 2015 at 11:37 AM, Jay Kreps <j...@confluent.io> wrote:
> Hey Todd,
>
> I actually agree on both counts.
>
> I would summarize the first comment as "Zookeeper is not hard to
> operationalize if you are Todd Palino"--also in that category of
> thing
[
https://issues.apache.org/jira/browse/KAFKA-2759?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14994590#comment-14994590
]
Todd Palino commented on KAFKA-2759:
The thought we had internally on the situation in KAFKA-1006
[
https://issues.apache.org/jira/browse/KAFKA-2235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14971806#comment-14971806
]
Todd Palino commented on KAFKA-2235:
I don't think we can. I have already increased it from 512MB
[
https://issues.apache.org/jira/browse/KAFKA-2235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14971740#comment-14971740
]
Todd Palino commented on KAFKA-2235:
I'm sure [~jjkoshy] will follow along with more detail
There seems to be a bit of detail lacking in the KIP. Specifically, I'd
like to understand:
1) What znodes are the brokers going to secure? Is this configurable? How?
2) What ACL is the broker going to apply? Is this configurable?
3) How will the admin tools (such as preferred replica election
> though.
>
> Thanks,
> -Flavio
>
> > On 21 Oct 2015, at 17:41, Todd Palino <tpal...@gmail.com> wrote:
> >
> > While this is a great idea, is it really ready for vote? I don't see any
> > detail in the wiki about what trees will be secured, and whether or
While this is a great idea, is it really ready for vote? I don't see any
detail in the wiki about what trees will be secured, and whether or not
that is configurable. I also don't see anything about how the use of admin
tools is going to be addressed.
-Todd
On Wed, Oct 21, 2015 at 8:48 AM, Grant
, Oct 21, 2015 at 9:56 AM, Flavio Junqueira <f...@apache.org> wrote:
>
> > On 21 Oct 2015, at 17:47, Todd Palino <tpal...@gmail.com> wrote:
> >
> > There seems to be a bit of detail lacking in the KIP. Specifically, I'd
> > like to understand:
> >
>
gree that eliminating a step
> > is an advantage, though.
> >
> > > Given we are assuming all the information in zookeeper is world
> readable
> > ,
> > > I don¹t see SSL support as a must have or a blocker for this KIP.
> >
> > OK, but k
On Wed, Oct 21, 2015 at 3:38 PM, Flavio Junqueira <f...@apache.org> wrote:
>
> > On 21 Oct 2015, at 21:54, Todd Palino <tpal...@gmail.com> wrote:
> >
> > Thanks for the clarification on that, Jun. Obviously, we haven't been
> doing
> > much with ZK auth
+1 (non-binding)
On Wed, Oct 21, 2015 at 6:53 PM, Jiangjie Qin
wrote:
> +1 (non-binding)
>
> On Wed, Oct 21, 2015 at 3:40 PM, Joel Koshy wrote:
>
> > +1 binding
> >
> > On Wed, Oct 21, 2015 at 8:17 AM, Flavio Junqueira
> wrote:
[
https://issues.apache.org/jira/browse/KAFKA-2580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14964003#comment-14964003
]
Todd Palino commented on KAFKA-2580:
It's about as graceful as an OOM, which is to say "not
[
https://issues.apache.org/jira/browse/KAFKA-2580?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14963986#comment-14963986
]
Todd Palino commented on KAFKA-2580:
I agree with [~jkreps] here, that having a high FD limit
[
https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14964228#comment-14964228
]
Todd Palino commented on KAFKA-2017:
I think we definitely need to maintain the ability to get
[
https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14960809#comment-14960809
]
Todd Palino commented on KAFKA-2017:
Just to throw in my 2 cents here, I don't think that persisting
We should also consider what else should be negotiated between the broker
and the client as this comes together. The version is definitely first, but
there are other things, such as the max message size, that should not need
to be replicated on both the broker and the client. Granted, max message
I tend to like the idea of a pluggable locator. For example, we already
have an interface for discovering information about the physical location
of servers. I don't relish the idea of having to maintain data in multiple
places.
-Todd
On Mon, Sep 28, 2015 at 4:48 PM, Aditya Auradkar <
+1000
!
-Todd
On Wednesday, September 23, 2015, Jiangjie Qin
wrote:
> Hi,
>
> Thanks a lot for the reviews and feedback on KIP-31. It looks all the
> concerns of the KIP has been addressed. I would like to start the voting
> process.
>
> The short summary for the
So, with regards to why you want to search by timestamp, the biggest
problem I've seen is with consumers who want to reset their timestamps to a
specific point, whether it is to replay a certain amount of messages, or to
rewind to before some problem state existed. This happens more often than
[
https://issues.apache.org/jira/browse/KAFKA-1566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14708282#comment-14708282
]
Todd Palino commented on KAFKA-1566:
This makes a lot of sense. We don't use any
It does not. That last connection, the one in CLOSE_WAIT, is an outbound
connection from the broker you are looking at to one of the other brokers.
57821 is the source TCP port, and it is selected (somewhat) randomly from a
range of high port numbers. Note that the other end of the connection is
data, and since we always encrypt compressed bits (compressing
encrypted bits doesn't compress), we need the broker to decrypt before
re-compressing.
On Fri, Jul 31, 2015 at 2:27 PM, Todd Palino tpal...@gmail.com wrote:
It does limit it to clients that have an implementation for encryption
It does limit it to clients that have an implementation for encryption,
however encryption on the client side is better from an auditing point of
view (whether that is SOX, HIPAA, PCI, or something else). Most of those
types of standards are based around allowing visibility of data to just the
Since I've been dealing with the fallout of this particular problem all
week, I'll add a few thoughts...
On Wed, Jul 22, 2015 at 10:51 AM, Gwen Shapira gshap...@cloudera.com
wrote:
On July 21, 2015, 11:15 p.m., Gwen Shapira wrote:
core/src/main/scala/kafka/network/SocketServer.scala,
.
On Fri, Jul 10, 2015 at 2:12 PM, Todd Palino tpal...@gmail.com wrote:
I absolutely disagree with #2, Neha. That will break a lot of
infrastructure within LinkedIn. That said, removing . might break
other
people as well, but I think we should have a clearer idea of how much
usage
I had to go look this one up again to make sure -
https://issues.apache.org/jira/browse/KAFKA-495
The only valid character names for topics are alphanumeric, underscore, and
dash. A period is not supposed to be a valid character to use. If you're
seeing them, then one of two things have happened:
than ASCII alphanumerics, '.', '_' and
'-')
case None = throw new InvalidTopicException(topic name + topic +
is illegal, contains a character other than ASCII alphanumerics, '.',
'_' and '-')
}
}
}
On Fri, Jul 10, 2015 at 2:50 PM, Todd Palino tpal...@gmail.com wrote:
I had
a character other than ASCII alphanumerics,
'.',
'_' and '-')
}
}
}
On Fri, Jul 10, 2015 at 2:50 PM, Todd Palino tpal...@gmail.com
wrote:
I had to go look this one up again to make sure -
https://issues.apache.org/jira/browse/KAFKA-495
The only valid character
than combining metrics for people who have a
'foo_bar' AND 'foo.bar' (and after all, having both is DEEPLY perverse,
no?).
Where is our Dean of Compatibility, Ewen, on this?
-Jay
On Fri, Jul 10, 2015 at 1:32 PM, Todd Palino tpal...@gmail.com
wrote:
My selfish point
/browse/KAFKA-697
On Fri, Jul 10, 2015 at 3:18 PM, Todd Palino tpal...@gmail.com wrote:
This was definitely changed at some point after KAFKA-495. The question
is
when and why.
Here's the relevant code from that patch
of Compatibility, Ewen, on this?
-Jay
On Fri, Jul 10, 2015 at 1:32 PM, Todd Palino tpal...@gmail.com wrote:
My selfish point of view is that we do #1, as we use _ extensively in
topic names here :) I also happen to think it's the right choice,
specifically because . has more special meanings
Congrats, Gwen! It's definitely deserved.
-Todd
On Jul 6, 2015, at 6:08 PM, Joe Stein joe.st...@stealth.ly wrote:
I am pleased to announce that the Apache Kafka PMC has voted to invite Gwen
Shapira as a committer and Gwen has accepted.
Please join me on welcoming and congratulating
[
https://issues.apache.org/jira/browse/KAFKA-2252?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14586437#comment-14586437
]
Todd Palino commented on KAFKA-2252:
I moved the normal connection closed message
makes sense if we plan to have more than two types
of
entity configs.. it's definitely more future proof. Thoughts?
Aditya
From: Todd Palino [tpal...@gmail.com]
Sent: Monday, May 18, 2015 12:39 PM
To: dev@kafka.apache.org
Subject: Re: [VOTE] KIP-21
I don't believe we're using the ResponsesBeingSent information at all. I
know we use NetworkProcessorAvgIdlePercent to keep track of the utilization
of the pool.
-Todd
On Thu, May 14, 2015 at 11:36 AM, Joel Koshy jjkosh...@gmail.com wrote:
I'm also not sure how useful it is, but there is some
I've been watching this discussion for a while, and I have to jump in and
side with Gwen here. I see no benefit to putting the configs into Zookeeper
entirely, and a lot of downside. The two biggest problems I have with this
are:
1) Configuration management. OK, so you can write glue for Chef to
[
https://issues.apache.org/jira/browse/KAFKA-2175?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Todd Palino updated KAFKA-2175:
---
Status: Patch Available (was: Open)
Reduce server log verbosity at info level
[
https://issues.apache.org/jira/browse/KAFKA-2175?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Todd Palino reassigned KAFKA-2175:
--
Assignee: Todd Palino (was: Neha Narkhede)
Reduce server log verbosity at info level
Todd Palino created KAFKA-2175:
--
Summary: Reduce server log verbosity at info level
Key: KAFKA-2175
URL: https://issues.apache.org/jira/browse/KAFKA-2175
Project: Kafka
Issue Type: Improvement
I tend to agree with Parth's point here. Most ACL systems I run into have deny
and allow. In general, you have a default policy of allow, then you follow your
rules stopping at the first line that matches. If you would like a default deny
policy, you have a bunch of allow rules and your last
a background copy of the log from the
current disk to the new disk, then it would take the partition offline and
delete the old log, then bring the partition back using the new log and
catch it back up off the leader.
-Jay
On Thu, Apr 9, 2015 at 8:19 AM, Todd Palino tpal...@gmail.com wrote:
I
I think this is a good start. We've been discussing JBOD internally, so
it's good to see a discussion going externally about it as well.
The other big blocker to using JBOD is the lack of intelligent partition
assignment logic, and the lack of tools to adjust it. The controller is not
smart
[
https://issues.apache.org/jira/browse/KAFKA-1342?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14484278#comment-14484278
]
Todd Palino commented on KAFKA-1342:
Bump
I think we need to revive this. We have
I agree with Jun here, that it would make it easier to do lag checking.
However, for individual checks it's really not that much trouble to do the
second request. If you're doing a lot of lag checking (like every consumer
and every topic) where the scale would start to make a difference, I would
I understand the desire to not bloat this one change with too much more
work, and it's a good change to start with. That said, I have one note on
your comments:
I don't agree with this because right now you get back the current state of
the partitions so you can (today) write whatever logic you
, Mar 10, 2015 at 10:12 AM, Todd Palino tpal...@gmail.com wrote:
Thanks, Jay. On the interface, I agree with Aditya (and you, I believe)
that we don't need to expose the public API contract at this time, but
structuring the internal logic to allow for it later with low cost is a
good idea
First, a couple notes on this...
3 - I generally agree with the direction of not pre-optimizing. However, in
this case I'm concerned about the calculation of the cost of doing plugins
now vs. trying to refactor the code to do it later. It would seem to me
that doing it up front will have less
Todd Palino created KAFKA-2012:
--
Summary: Broker should automatically handle corrupt index files
Key: KAFKA-2012
URL: https://issues.apache.org/jira/browse/KAFKA-2012
Project: Kafka
Issue Type
to another, will we end up with some
unread messages hanging around and no one thinks or knows it is their
responsibility to take care of them?
Thanks.
Tong
Sent from my iPhone
On Mar 5, 2015, at 10:46 AM, Todd Palino tpal...@gmail.com wrote:
Apologize for the late comment
Apologize for the late comment on this...
So fair assignment by count (taking into account the current partition
count of each broker) is very good. However, it's worth noting that all
partitions are not created equal. We have actually been performing more
rebalance work based on the partition
Todd Palino created KAFKA-1987:
--
Summary: Potential race condition in partition creation
Key: KAFKA-1987
URL: https://issues.apache.org/jira/browse/KAFKA-1987
Project: Kafka
Issue Type: Bug
version is
probably better. It just means that we need to document the migration path
for previous versions.
Thanks,
Jun
On Wed, Feb 11, 2015 at 6:33 PM, Todd Palino tpal...@gmail.com wrote:
Thanks, Gwen. This looks good to me as far as the wire protocol
versioning
goes. I agree
Thanks, Gwen. This looks good to me as far as the wire protocol versioning
goes. I agree with you on defaulting to the new wire protocol version for
new installs. I think it will also need to be very clear (to general
installer of Kafka, and not just developers) in documentation when the wire
Leaving aside the rest of this, on #1, while I consider being able to
advertise the ports a good idea, I don't want to lose the ability for
maintaining multiple ports with the same protocol. For example, being able
to have 2 plaintext ports, one that only brokers communicate over, and one
that
think a bit on how this assumption can be relaxed.
Gwen
On Tue, Dec 2, 2014 at 3:14 PM, Todd Palino tpal...@gmail.com wrote:
Gwen - just my reading of what we could expect from what you had
described
so far. Without having gone into implementation details, there didn't
seem
?
Thanks,
Jun
On Tue, Dec 2, 2014 at 3:31 PM, Todd Palino tpal...@gmail.com wrote:
Thanks. Just to add more detail as to why I think it's a good idea to be
able to segregate traffic like that...
One reason would just be to separate out the intra-cluster communication
to
a separate port
For the moment, consumers still need to write under the /consumers tree.
Even if they are committing offsets to Kafka instead of ZK, they will need
to write owner information there when they are balancing. Eventually, you
are correct, this is going away with the new consumer.
-Todd
On Fri, Oct
1 - 100 of 122 matches
Mail list logo