[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2015-08-20 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14705068#comment-14705068
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Hi,

I noticed that the dependencies are done and I will resume this task.
The task contributions had been:
- a fix
- unit test(s)

As far as the fix is concerned, I noticed that it is already fixed in the 
current Selector, namely the lruConnections is a LinkedHashMap with 
accessOrder=true. This was the only fix needed, and I am 100% convinced that 
the fix is already done.

I already have a unit test too, I will try to put a patch here this week.

Just wanted to mention that the old connections should be closed by the kafka 
installations using the new reusable network code.

Thanks
Nicu

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2.0
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Attachments: 1282_access-order_+_test_(same_class).patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> access-order_+_test.patch, 
> access_order_+_test_waiting_from_350ms_to_1100ms.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2015-06-01 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14568257#comment-14568257
 ] 

Jun Rao commented on KAFKA-1282:


[~nmarasoi], thanks for the patch. We are changing SocketServer to reuse 
Selector right now in KAFKA-1928. Once that's done, the idle connection logic 
will be moved into Selector and should be easier to test since Selector 
supports mock time. That patch is almost ready. Perhaps you can wait until it's 
committed and submit a new patch. 

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2.0
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.8.3
>
> Attachments: 1282_access-order_+_test_(same_class).patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> access-order_+_test.patch, 
> access_order_+_test_waiting_from_350ms_to_1100ms.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-12-29 Thread Nicolae Marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14260516#comment-14260516
 ] 

Nicolae Marasoiu commented on KAFKA-1282:
-

I will do unit tests tommorow / day after. The fix should be ok otherwise,
and ready to be pushed on trunk and 0.8.2. I will announce when done with
units.

On Tue, Dec 30, 2014 at 12:21 AM, Neha Narkhede (JIRA) 



> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.8.2
>
> Attachments: 1282_access-order.patch, 1282_brush.patch, 
> 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-12-29 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14260486#comment-14260486
 ] 

Neha Narkhede commented on KAFKA-1282:
--

[~nmarasoi], [~junrao] This is marked for 0.8.2. Is anyone working or planning 
to work on this?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.8.2
>
> Attachments: 1282_access-order.patch, 1282_brush.patch, 
> 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-11-12 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14208810#comment-14208810
 ] 

nicu marasoiu commented on KAFKA-1282:
--

I want, yes, I will add a few tests this week.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_access-order.patch, 1282_brush.patch, 
> 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-11-10 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14205156#comment-14205156
 ] 

Jun Rao commented on KAFKA-1282:


Nicu,

Thanks for the patch. Do you think it's easy to add a unit test on Processor?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_access-order.patch, 1282_brush.patch, 
> 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-11-09 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14204246#comment-14204246
 ] 

Neha Narkhede commented on KAFKA-1282:
--

good catch [~nmarasoi]. +1 on your change

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_access-order.patch, 1282_brush.patch, 
> 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-11-08 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14203456#comment-14203456
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Indeed, I can reproduce this. I did saw an instance where no exception was 
thrown by the producer but still the broker mentioned new connection being 
listened to suggesting close took place. However, checking with required-acks 0 
I can see that after some time the connection does not close anymore.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brush.patch, 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-11-08 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14203334#comment-14203334
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Hi,

>From my experiments, I noticed a new connection is always spawned after the 
>timeout, even if the producer does not get an exception. That certainly means 
>the old connection would have been closed.
I am looking now why is it possible that the producer is not getting any 
exception but a more relevant test would be without the required-acks I think.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brush.patch, 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-11-06 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14201475#comment-14201475
 ] 

Jun Rao commented on KAFKA-1282:


Nicu,

I was doing some manual testing of this feature. What I observed is that 
sometimes, the idle connections are not closed. The following was what I did.

1. Configure a small connections.max.idle.ms = 1.
2. start ZK and Kafka broker
3. start a console consumer
bin/kafka-console-consumer.sh --zookeeper localhost:2181 --topic topic1 
--from-beginning
4. start a console producer and type in sth every 15 secs or so. 
bin/kafka-console-producer.sh --broker-list localhost:9092 --topic topic1 
--request-required-acks 1

What I observed was that initially, the producer connections kept getting 
killed by the broker correctly after being idle for 10 secs. The next producer 
send would hit an IOException and trigger a resend. However, after typing in 10 
or so messages, at some point, no idle connections were killed by the broker 
any more and the producer send always succeeded.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brush.patch, 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-20 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14142199#comment-14142199
 ] 

Neha Narkhede commented on KAFKA-1282:
--

Pushed the latest patch to trunk.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brush.patch, 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-20 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14142174#comment-14142174
 ] 

Neha Narkhede commented on KAFKA-1282:
--

+1 on your latest patch. I'm leaning towards accepting the patch since the test 
above points to an issue that seems unrelated to the patch. [~nmarasoi], it 
will be great if you can follow Jun's suggestion to reproduce the issue. Then 
file a JIRA to track it. I'm guessing killing idle connections shouldn't lead 
to data loss.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brush.patch, 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-18 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14139513#comment-14139513
 ] 

Jun Rao commented on KAFKA-1282:


Do you think you can reproduce that data loss issue in 1 out of your 7 tests? 
With ack=1 and retries, this shouldn't happen. Perhaps it's useful to enable 
the trace logging in the producer to see what's exactly happening there.

Could you also do the same test by enabling the new producer in console 
producer?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brush.patch, 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-18 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14138972#comment-14138972
 ] 

Neha Narkhede commented on KAFKA-1282:
--

Thanks for the updated patch. Overall, looks great. Few comments -
1. Can you rename initialNextIdleCloseCheckTimeValue to 
nextIdleCloseCheckTimeValue?
2. It will be easier to understand the code if we rename currentTime to 
currentTimeNanos.


> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-17 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14137796#comment-14137796
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Indeed, ack=1 solves it, it gets a reset by peer and a socket timeout on fetch 
meta, than re connects and sends message.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-17 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14137399#comment-14137399
 ] 

Jun Rao commented on KAFKA-1282:


Interesting. The data loss may have to do with ack=0, which is the default in 
console producer. Could you try ack=1?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-17 Thread Nicolae Marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14137130#comment-14137130
 ] 

Nicolae Marasoiu commented on KAFKA-1282:
-

in fact, this is something that needs fixing in the producer(s) anyway, but the 
issue is with the currently deployed producers.
One of the main reasons to go with a broker side close of the idle connections 
was that it is easier to redeploy brokers then producers.
But if this is indeed a bug in the producer(s) as I reproduced, those producers 
would need redeploy.
So moving this to the producer side as a configuration may again be an option 
on the table.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_brushed_up.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-17 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14137118#comment-14137118
 ] 

nicu marasoiu commented on KAFKA-1282:
--

here is a time line:

he -> produced
he -> consumed
[ wait beyond timeout here, connection got closed underneath by the other side]
[2014-09-17 15:02:28,689] INFO Got user-level KeeperException when processing 
sessionid:0x148837ce181 type:setData cxid:0x24 zxid:0xec txntype:-1 
reqpath:n/a Error Path:/consumers/console-consumer-87959/offsets/topi/0 
Error:KeeperErrorCode = NoNode for 
/consumers/console-consumer-87959/offsets/topi/0 
(org.apache.zookeeper.server.PrepRequestProcessor)
[2014-09-17 15:02:28,691] INFO Got user-level KeeperException when processing 
sessionid:0x148837ce181 type:create cxid:0x25 zxid:0xed txntype:-1 
reqpath:n/a Error Path:/consumers/console-consumer-87959/offsets 
Error:KeeperErrorCode = NoNode for /consumers/console-consumer-87959/offsets 
(org.apache.zookeeper.server.PrepRequestProcessor)
dd --> produce attempt (never retried, or never reached the broker 
or at least never reached the consumer)
[ many seconds wait, to see if the message is being retried, apparently not, 
even though the default retry is 3 times]
w --> new attempt (immediattely I see the message below with 
the stack trace, and reconnect + retry is instantly sucesfull)
[2014-09-17 15:03:12,599] WARN Failed to send producer request with correlation 
id 9 to broker 0 with data for partitions [topi,0] 
(kafka.producer.async.DefaultEventHandler)
java.io.IOException: Broken pipe
at sun.nio.ch.FileDispatcherImpl.writev0(Native Method)
at sun.nio.ch.SocketDispatcher.writev(SocketDispatcher.java:51)
at sun.nio.ch.IOUtil.write(IOUtil.java:149)
at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:483)
at java.nio.channels.SocketChannel.write(SocketChannel.java:493)
at 
kafka.network.BoundedByteBufferSend.writeTo(BoundedByteBufferSend.scala:56)
at kafka.network.Send$class.writeCompletely(Transmission.scala:75)
at 
kafka.network.BoundedByteBufferSend.writeCompletely(BoundedByteBufferSend.scala:26)
at kafka.network.BlockingChannel.send(BlockingChannel.scala:100)
at kafka.producer.SyncProducer.liftedTree1$1(SyncProducer.scala:72)
at 
kafka.producer.SyncProducer.kafka$producer$SyncProducer$$doSend(SyncProducer.scala:71)
at 
kafka.producer.SyncProducer$$anonfun$send$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(SyncProducer.scala:102)
at 
kafka.producer.SyncProducer$$anonfun$send$1$$anonfun$apply$mcV$sp$1.apply(SyncProducer.scala:102)
at 
kafka.producer.SyncProducer$$anonfun$send$1$$anonfun$apply$mcV$sp$1.apply(SyncProducer.scala:102)
at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33)
at 
kafka.producer.SyncProducer$$anonfun$send$1.apply$mcV$sp(SyncProducer.scala:101)
at 
kafka.producer.SyncProducer$$anonfun$send$1.apply(SyncProducer.scala:101)
at 
kafka.producer.SyncProducer$$anonfun$send$1.apply(SyncProducer.scala:101)
at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33)
at kafka.producer.SyncProducer.send(SyncProducer.scala:100)
at 
kafka.producer.async.DefaultEventHandler.kafka$producer$async$DefaultEventHandler$$send(DefaultEventHandler.scala:255)
at 
kafka.producer.async.DefaultEventHandler$$anonfun$dispatchSerializedData$2.apply(DefaultEventHandler.scala:106)
at 
kafka.producer.async.DefaultEventHandler$$anonfun$dispatchSerializedData$2.apply(DefaultEventHandler.scala:100)
at 
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772)
at 
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
at 
scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
at 
scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
at 
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
at 
kafka.producer.async.DefaultEventHandler.dispatchSerializedData(DefaultEventHandler.scala:100)
at 
kafka.producer.async.DefaultEventHandler.handle(DefaultEventHandler.scala:72)
at 
kafka.producer.async.ProducerSendThread.tryToHandle(ProducerSendThread.scala:104)
at 
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:87)
at 
kafka.producer.async.ProducerSendThread$$anonfun$processEvents$3.apply(ProducerSendThread.scala:67)
at scala.collection.immutable.Stream.foreach(Stream.scala:547)
at 
kafka.producer.async.ProducerSendThread.processEvents(ProducerSendThread.scala:66)
at 

[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-17 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14137011#comment-14137011
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Hi,

Unfortunately the client used in console-producer is not robust with respect to 
disconnections, as below. Is this the "old" scala producer, and can we hope for 
a resilient behaviour that I can test with the new java producer?

[2014-09-17 12:44:12,009] WARN Failed to send producer request with correlation 
id 15 to broker 0 with data for partitions [topi,0] 
(kafka.producer.async.DefaultEventHandler)
java.io.IOException: Broken pipe
at sun.nio.ch.FileDispatcherImpl.writev0(Native Method)
at sun.nio.ch.SocketDispatcher.writev(SocketDispatcher.java:51)
at sun.nio.ch.IOUtil.write(IOUtil.java:149)
at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:483)
at java.nio.channels.SocketChannel.write(SocketChannel.java:493)
at 
kafka.network.BoundedByteBufferSend.writeTo(BoundedByteBufferSend.scala:56)
at kafka.network.Send$class.writeCompletely(Transmission.scala:75)
at 
kafka.network.BoundedByteBufferSend.writeCompletely(BoundedByteBufferSend.scala:26)
at kafka.network.BlockingChannel.send(BlockingChannel.scala:100)


> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_the_ultimate,_close_max_one_per_select.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-16 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14136767#comment-14136767
 ] 

Jun Rao commented on KAFKA-1282:


Did you do "./gradlew jar" first?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282_the_ultimate,_close_max_one_per_select.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-16 Thread Nicolae Marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14135561#comment-14135561
 ] 

Nicolae Marasoiu commented on KAFKA-1282:
-

Hi, I have not made any patch yet, I waited for feedback from Neha too, but I 
will do the patch today, it looks ok to me the idea of closing at most one old 
connection per selector iteration.

For #1, the way Neha and me discussed, and the way you understood it works (for 
the latest patch), is that an old connection is taken into consideration for 
close only when a new connection is being opened up (or activity exists on an 
existing connection too).

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-15 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14134947#comment-14134947
 ] 

Jun Rao commented on KAFKA-1282:


Nicu,

On #2, I wasn't worried about any performance optimization. My concern is 
mostly on testing and ease of understanding. Since removeEldestEntry is only 
called on update, you can't test the logic on a single connection to the 
broker. It's a bit weird that if there is only a single idle connection, that 
connection is never killed. But as soon as a second connection is added, the 
idle connection will be killed. For the user's perspective, it's simpler to 
understand how idle connections are killed if they are not tied to # of 
connection.

Also, could you explain how you fixed #1 in the latest patch? It wasn't obvious 
to me.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-14 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14133259#comment-14133259
 ] 

Neha Narkhede commented on KAFKA-1282:
--

Thanks for the patch, [~nmarasoi]! Looks good overall. Few review comments -

1. Do we really need connectionsLruTimeout in addition to connectionsMaxIdleMs? 
It seems to me that we are translating the idle connection timeout plugged in 
by the user to 100x times more than what is configured. That's probably why 
Jun saw the behavior he reported earlier. 
2. I don't really share Jun's concern in #2 and we can state that more clearly 
in the comment that describes the new config in KafkaConfig. Connections that 
are idle for more than connections.max.idle.ms *may* get killed. I don't think 
the users particularly care about a hard guarantee of their connections getting 
killed here. So the simplicity of this approach is well justified.
3. I do think that adding a produce and fetch test where the connections get 
killed will be great 

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-08 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14125466#comment-14125466
 ] 

nicu marasoiu commented on KAFKA-1282:
--

[~nehanarkhede] Hi, can you also check the new idea? It is consistent with my 
initial approach and solves the potential overhead of closing too many 
connections on a single iteration.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-08 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14125465#comment-14125465
 ] 

nicu marasoiu commented on KAFKA-1282:
--

[~junrao], hi, can you answer please? I agree with what you say if I understood 
all of it, I am doing a small patch right now

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-04 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14122456#comment-14122456
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Hi, I am not completely sure I fully understood your solution in point 2: 

Do you mean to close at most one connection per iteration, right? This is ok, 
the worst case scenario is closing 100K old connections in 10 hours, one per 
select.

On storing the time to close in a local variable, the access of the oldest 
entry every iteration is O(1) super cheap so I would skip this optimization. 

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-02 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14118317#comment-14118317
 ] 

Jun Rao commented on KAFKA-1282:


Thanks for the latest patch. I was trying to do some local testing. The 
following are my observations.

1. I first started a local ZK and broker (setting connections.max.idle.ms 
10secs). I then started a console-producer and a console-consumer. Then, I 
typed in sth in console-producer every 15 secs. However, I don't see the 
producer connection gets killed. I added sth instrumentation. It doesn't seem 
that removeEldestEntry() is called on every fetch request.

2. As I was debugging this, I realized that it's kind of weird to kill idle 
connections only when there is another non-idle connection. This makes 
debugging harder since one can't just test this out with a single connection. 
It's much simpler to understand if the idle connection can just be killed after 
the connection idle time, independent of other connections to the broker. To 
address the concern of closing many sockets in one iteration of the selector, 
we can calculate the time that a socket entry is expected to be killed (this is 
the access time of the oldest entry + maxIdleTime, or maxIdleTime if no entry 
exists). When that time comes during the iteration of the selector, we can just 
check the oldest entry and see if it needs to be closed.

3. It would be good to check if our clients (especially the producer, both old 
and new) can handle a closed idle connection properly. For example, when 
detecting an already closed socket, the producer should be able to resend the 
message and therefore we shouldn't see any data loss.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> idleDisconnect.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-09-01 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14117621#comment-14117621
 ] 

Jun Rao commented on KAFKA-1282:


Looking at the patch again, in removeEldestEntry(), shouldn't we close the 
socket for eldest if the entry is to be removed? Right now, it seems that we 
only remove the entry from LRU w/o actually closing the idle socket connection.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> Patch_updated__Configurable_max_idleness_of_a_connection_since_the_last_read_on_it__On_cre.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-28 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14114390#comment-14114390
 ] 

Jun Rao commented on KAFKA-1282:


Thanks for the patch. The following should be * 100, right?

  private val connectionsLruTimeout: Long = connectionsMaxIdleMs * 1000

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-28 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14114248#comment-14114248
 ] 

nicu marasoiu commented on KAFKA-1282:
--

[~nehanarkhede] Hi, I implemented our discussion and applied Jun Rao 
suggestions, can you check and perhaps commit it if looks good? Hope for more 
tasks like this, do you have any suggestions?:)

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 1282.patch, 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-26 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14110753#comment-14110753
 ] 

Jun Rao commented on KAFKA-1282:


Nicu,

Similar to producers, consumers just issue fetch requests. The SocketServer 
first reads the fetch request from the network and then writes the fetch 
response to the network once the fetch request is served by the broker. So, 
there is a 1-to-1 mapping btw reads and writes and writes typically happen 
within a second after the reads. 

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-25 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14110261#comment-14110261
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Hi, Thank you, for 2. I agree for producers but I am not sure if the same 
SocketServer is used to serve consumers as well, and in this case, for 
consumers, the read/write ratio may be well in favor of writes making it risky 
perhaps to account just the reads?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-24 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14108716#comment-14108716
 ] 

Jun Rao commented on KAFKA-1282:


Thanks for the patch. Looks good to me overall. Some minor comments below.

1. Could we make connectionsLruTimeout a broker side configuration?

2. Do we need to insert the key to lruConnections in write()? It seems to me 
doing that in read() (for incoming requests) is enough.

3. The patch doesn't seem to apply for me. Could you rebase?

git  apply -p0 
~/Downloads/KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch 
/Users/jrao/Downloads/KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch:13:
 trailing whitespace.
import java.util
/Users/jrao/Downloads/KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch:21:
 trailing whitespace.
import java.util.Map.Entry
/Users/jrao/Downloads/KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch:30:
 trailing whitespace.
  private val connectionsLruTimeout: Long = TimeUnit.MINUTES.toNanos(10)
/Users/jrao/Downloads/KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch:31:
 trailing whitespace.
  private var currentTime: Long = SystemTime.nanoseconds
/Users/jrao/Downloads/KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch:32:
 trailing whitespace.
  private val lruConnections = new util.LinkedHashMap[SelectionKey, Long](100, 
.75F, true) {
error: patch failed: core/src/main/scala/kafka/network/SocketServer.scala:16
error: core/src/main/scala/kafka/network/SocketServer.scala: patch does not 
apply


> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> KAFKA-1282_Disconnect_idle_socket_connection_in_Selector.patch, 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-21 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14105606#comment-14105606
 ] 

nicu marasoiu commented on KAFKA-1282:
--

After discussion with Neha, we agreed that using the removeEldestEntry approach 
works better in the sense that avoids disruption caused by potentially many 
connections being up for close at once, and evens out that overhead. The 
disadvantage remains that an inactive server will not close connections but 
seems less than the advantage of closing overhead leveling and of performance 
plus of not traversing and of not polling the oldest entry.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-15 Thread Nicolae Marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14098324#comment-14098324
 ] 

Nicolae Marasoiu commented on KAFKA-1282:
-

Hi, I am sorry, but traversing will be limited to the connections that will 
actually be expired, so there is no traversing of non-expiring connections!
Say the server hold 100K connections. Say 100 connections are not used in the 
last 10 minutes.
What the program does (or I will make sure it does) is just iterate through the 
first 101 connections, the first 100 will be expired and it will stop at number 
101.
I think this is an exact achievement of expected behavior of the jira task, as 
intended, and there is no performance penalty to that really!

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-14 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14097971#comment-14097971
 ] 

Neha Narkhede commented on KAFKA-1282:
--

My suggestion was not just to address the performance concern which is somewhat 
of an issue nevertheless. The motivation was that there is an upper bound on 
the number of open connections you can support on the broker. That number is 
the # of open file handles configured on the box. Since that number is known 
anyway, you probably would want to configure your server so that the 
connections never exceed a certain percentage of that upper limit. Currently, 
if the server runs out of open file handles, it effectively stays alive, but is 
unable to serve any data and becomes a 'zombie'. 

But a downside of the expiration based on the connection count is that it 
doesn't necessarily achieve the goal of expiring really old connections. 
Instead it tries to solve the problem of preventing the broker from running out 
of available file handles, in which case we probably need a fairer strategy for 
expiring connections. 

Thinking more, I think it might be sufficient to override removeEldestEntry and 
check if the oldest entry is older than the threshold and let the map remove 
it. If the oldest entry is not above the threshold, traversing the map doesn't 
buy you anything. The downside is that if no new activity takes place on any of 
the connections all of a sudden, the server wouldn't proactively drop all 
connections, which is less of a concern. 

The advantage is that you will still get the same benefit of expiring older 
connections and it removes the need to traverse.


> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-14 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14097380#comment-14097380
 ] 

nicu marasoiu commented on KAFKA-1282:
--

To make the ~O(1) cost of "traversing" more clear, typically only the first 
element in the linked list is accessed, and it will typically be used in the 
last 10 minutes, and in this case nothing happens anymore. Of course, this is 
if the low volume topics do not generate many connections, which they won't, 
with this cleaning up in place. And I am checking now that map() and the rest 
are lazy, or else for sure I can make so that only the relevant "prefix/first" 
part of the collection is iterated, typically first element only.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-14 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14097047#comment-14097047
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Traversing is quite cheap and can be done every 1000 selects.
The intent of your suggestion is to optimize, I understand, but the effects is 
a different behavior as I feel it (changes the expiration by time and switches 
it to an expiration by connection count), and to a low performance benefit (I 
think traversing is much cheaper than blocking close on each channel, that 
would happen either way).
The idea of limited connection count can be used complementary to the existing 
traversing, but if you mean to take out the traversing every n selects, that 
changes the expiration by time and switches it to an expiration by connection 
count - is it an agreed requirements change with [~junrao]? I must warn that it 
is dangerous in my view to configure a maximum connection count per broker, 
because in event many brokers go down, and many clients need to use the system, 
this connection thrashing would not help anybody, and be a worse effect than 
not having this connection expiration at all, in such a scenario, relevant to a 
highly available system.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-14 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14097029#comment-14097029
 ] 

Neha Narkhede commented on KAFKA-1282:
--

Took a look at the patch. How about the following -
1. Limit the LRU cache size to the number of active connections that should be 
supported by the Kafka server. I'm guessing this should be a config. 
2. Override removeEldestEntry to evict the oldest entry if the cache size 
exceeds the configured number of LRU connections.

That way, we don't have to traverse the map several times in the main loop, 
which can be expensive.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-14 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14097012#comment-14097012
 ] 

Neha Narkhede commented on KAFKA-1282:
--

Thanks for picking this up [~nmarasoi]. Assigning to myself for review. 

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: Neha Narkhede
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-12 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14094686#comment-14094686
 ] 

nicu marasoiu commented on KAFKA-1282:
--

I attached a first version of the patch.
I am still thinking on any other implications, but wanted to share a first 
draft to collect some feedback already.
Thanks

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
> Attachments: 
> kafka-1282__Disconnect_idle_socket_connection_in_Selector.patch
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-08 Thread nicu marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14090832#comment-14090832
 ] 

nicu marasoiu commented on KAFKA-1282:
--

Hi,

I will spend 1-2 hours per day the next week.
So I would like to keep this nice task.
My estimate, I will have a first working solution to put up for review in ~3 
days, so likely Thursday.

Does that sound good?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-08-06 Thread Neha Narkhede (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14087676#comment-14087676
 ] 

Neha Narkhede commented on KAFKA-1282:
--

Hey [~nmarasoiu], are you actively working on this patch yet? If not, do you 
mind if we have someone else pick it up?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-07-23 Thread Jun Rao (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14071803#comment-14071803
 ] 

Jun Rao commented on KAFKA-1282:


Yes. Thanks for picking it up.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-07-23 Thread Nicolae Marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14071718#comment-14071718
 ] 

Nicolae Marasoiu commented on KAFKA-1282:
-

[~junrao] You agree with the approach, do you?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>Assignee: nicu marasoiu
>  Labels: newbie++
> Fix For: 0.9.0
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-07-18 Thread Nicolae Marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14066481#comment-14066481
 ] 

Nicolae Marasoiu commented on KAFKA-1282:
-

Beautiful, I can't wait to work this out, so I take this to code right?:)


> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>  Labels: newbie++
> Fix For: 0.9.0
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-07-18 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14066461#comment-14066461
 ] 

Jay Kreps commented on KAFKA-1282:
--

The goal is just to reduce server connection count. In our environment there 
might be a single Kafka producer in each process we run publishing to a small 
Kafka cluster (say ~20 servers). However there are tens of thousands of client 
processes. Connections can end up going unused when leadership migrates and we 
should eventually close these out rather than retaining them indefinitely.

As you say it is not critical as the server seems to do a good job of dealing 
with high connection counts, but it seems like a good thing to do.

I agree that doing this on the server might be better. This does mean it is 
possible that the server will attempt to close the socket while the client is 
attempting to send something. But if the timeout is 10 mins, it is unlikely 
that this will happen often (i.e. if nothing was sent in the last 10 mins, it 
will not likely happen in the 0.5 ms it takes to do the close). The advantage 
of doing it on the server is that it will work for all clients.

This change would be in core/.../kafka/network/SocketServer.scala.

The only gotcha is that we likely need to avoid iterating over all connections 
to avoid latency impact (there could be 100k connections). One way to do this 
would be to use java.util.LinkedHashMap to implement an LRU hash map of the 
SelectionKeys, and access this every time the selection key comes up in a 
select operation. (There are a ton of details in LinkedHashMap--needs to be 
"access order", etc). Then every 5-10 select loop iterations we would iterate 
the map expiring connections until we come to a connection that doesn't need 
expiring, then stop.

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>  Labels: newbie++
> Fix For: 0.9.0
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (KAFKA-1282) Disconnect idle socket connection in Selector

2014-07-18 Thread Nicolae Marasoiu (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-1282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14066315#comment-14066315
 ] 

Nicolae Marasoiu commented on KAFKA-1282:
-

Right, the limitation is more critical on the client side of a client-server 
connection due to port count limitation, and/or socket/file count restrictions 
of the client env.

On the other hand, the brokers could close the connections too on such 
condition, rather than relying on the clients(producers) to protect it.

However, what is any other reason to reduce the socket connections count? To 
make the NIO select lighter on the server, on a lesser number of connections? I 
think epoll is quite relaxed on this.

I would like to work on this, but also understand the original problem(s) / 
concern(s) to see if we can also see any more suitable solutions to the 
particular concern?

> Disconnect idle socket connection in Selector
> -
>
> Key: KAFKA-1282
> URL: https://issues.apache.org/jira/browse/KAFKA-1282
> Project: Kafka
>  Issue Type: Bug
>  Components: producer 
>Affects Versions: 0.8.2
>Reporter: Jun Rao
>  Labels: newbie++
> Fix For: 0.9.0
>
>
> To reduce # socket connections, it would be useful for the new producer to 
> close socket connections that are idle. We can introduce a new producer 
> config for the idle time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)