Hi Jay,
      Thanks for the review. 

   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 the 
network threads immediately? I guess I would have to look at that code to 
know... 
        I’ve non-blocking handshake on the server side as well as for new 
producer client.  Blocking handshake is only done for BlockingChannel.scala and 
it just loops over the non-blocking hand shake until the context is 
established. So on the server side (SocketServer.scala) as it goes through the 
steps and returns “READ or WRITE” signal for next step.  For BlockingChannel 
the worst case I look at is the connection timeout but most times this 
handshake will finish up much quicker . I am cleaning up the code will send up 
a patch in next few days .

2. Do we need to support blocking channel at all? That is just for the old 
clients, and I think we should probably just leave those be to reduce scope 
here. 
So blocking channel used not only by simple consumer but also 
ControllerChannelManager and controlled shutdown also. Are we planning on 
deprecating it. I think at least for ControllerChannelManager it makes sense  
to have a blocking channel. If the users want to lock down the cluster i.e no 
PLAINTEXT channels are allowed than all the communication has to go through 
either SSL and KERBEROS so in this case we need add this capability to 
BlockingChannel.



3. Can we change the APIs to drop the getters when that is not required by 
the API being implemented. In general we don't use setters and getters as a 
naming convention. 

My bad on adding getters and setters :). I’ll work on removing it and change 
the KIP accordingly. I still need some accessor methods though .

Thanks,

Harsha



On April 21, 2015 at 2:51:15 PM, Jay Kreps (jay.kr...@gmail.com) wrote:

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 the  
network threads immediately? I guess I would have to look at that code to  
know...  

2. Do we need to support blocking channel at all? That is just for the old  
clients, and I think we should probably just leave those be to reduce scope  
here.  

3. Can we change the APIs to drop the getters when that is not required by  
the API being implemented. In general we don't use setters and getters as a  
naming convention.  

The long explanation on that is that setters/getters kind of imply a style  
of java programming where you have simple structs with getters and setters  
for each field. In general we try to have access methods only when  
necessary, and rather than setters model the full change or action being  
carried out, and if possible disallow change entirely. This is more in line  
with modern java style I think. We aren't perfect in following this, but  
once you start with getters and setters people start just adding them  
everywhere and then using them.  

-Jay  


On Mon, Apr 20, 2015 at 10:42 AM, Sriharsha Chintalapani <ka...@harsha.io>  
wrote:  

> Hi,  
> I updated the KIP-12 with more details. Please take a look  
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=51809888  
>  
> Thanks,  
> Harsha  
>  
>  
> On February 11, 2015 at 10:02:43 AM, Harsha (ka...@harsha.io) wrote:  
>  
> Thanks Joe. It will be part of KafkaServer and will run on its own  
> thread. Since each kafka server will run with a keytab we should make  
> sure they are all getting renewed.  
>  
> On Wed, Feb 11, 2015, at 10:00 AM, Joe Stein wrote:  
> > Thanks Harsha, looks good so far. How were you thinking of running  
> > the KerberosTicketManager as a standalone process or like controller or  
> > is  
> > it a layer of code that does the plumbing pieces everywhere?  
> >  
> > ~ Joestein  
> >  
> > On Wed, Feb 11, 2015 at 12:18 PM, Harsha <ka...@harsha.io> wrote:  
> >  
> > > Hi,  
> > > Here is the initial proposal for sasl/kerberos implementation for  
> > > kafka https://cwiki.apache.org/confluence/x/YI4WAw  
> > > and JIRA https://issues.apache.org/jira/browse/KAFKA-1686. I am  
> > > currently working on prototype which will add more details to the KIP.  
> > > Just opening the thread to say the work is in progress. I'll update the  
> > > thread with a initial prototype patch.  
> > > Thanks,  
> > > Harsha  
> > >  
>  

Reply via email to