Re: [DISCUSS] KIP-248 Create New ConfigCommand That Uses The New AdminClient

2018-02-19 Thread Viktor Somogyi
Hi Rajini,

Thanks for the feedback, I've applied your points.

Viktor

On Wed, Feb 7, 2018 at 7:22 PM, Rajini Sivaram 
wrote:

> Hi Viktor,
>
> Thanks for the updates. Looks good, just a few minor comments:
>
>1. AdminOperation - could be AlterOperation since it is only applied to
>'Alter'?
>2. Don't think we need `Unknown` type to process old requests. We can
>use `Set` as the default for alter requests with version 0.
>3. There is a typo in
>https://cwiki.apache.org/confluence/display/KAFKA/KIP-248+-+Create+New+
> ConfigCommand+That+Uses+The+New+AdminClient#KIP-248-
> CreateNewConfigCommandThatUsesTheNewAdminClient-AdminClientAPIs
> :
>AdminOperation enum has a constructor QuotaType.
>
>
> On Wed, Feb 7, 2018 at 4:53 PM, Viktor Somogyi 
> wrote:
>
> > Hi Rajini,
> >
> > I think it makes sense absolutely and even we could do it for AlterQuotas
> > as we will have the same problem there.
> > Updated my KIP
> >  > ConfigCommand+That+Uses+The+New+AdminClient>
> > to
> > reflect these changes:
> > - proposed protocol changes
> > - created a AdminOperation type to represent the Add/Set/Delete triplet.
> > (Put in the org.apache.kafka.clients.admin package)
> >
> > Please let me know if I missed something that you thought otherwise.
> >
> > Regards,
> > Viktor
> >
> >
> > On Tue, Feb 6, 2018 at 1:31 PM, Rajini Sivaram 
> > wrote:
> >
> > > Hi Viktor,
> > >
> > > While implementing KAFKA-6494, I realised that there is a mismatch
> > between
> > > the --alter command of ConfigCommand and AlterConfigs request.
> > > ConfigCommand uses --add-config and --delete-config to make incremental
> > > updates. --add-config reads all the configs from ZooKeeper and adds the
> > > delta provided on top of that. AlterConfigs request currently sets the
> > > whole properties object, so you need to know the full set of properties
> > of
> > > an entity to use AlterConfigs request through the AdminClient. We don't
> > > allow sensitive configs to be read using AdminClient, so we can't read
> > and
> > > add configs as we do with ZooKeeper. So we need a protocol change to
> make
> > > this work. I didn't want to make this change after KIP freeze, so
> perhaps
> > > we could include this in your KIP? We could perhaps add a mode
> > > (SET/ADD/DELETE) for AlterConfigs request where SET matches the
> existing
> > > behaviour for backward compatibility and ConfigCommand uses ADD/DELETE.
> > >
> > > Thoughts?
> > >
> > > Regards,
> > >
> > > Rajini
> > >
> > > On Fri, Jan 19, 2018 at 12:57 PM, Viktor Somogyi <
> > viktorsomo...@gmail.com>
> > > wrote:
> > >
> > > > Hi Rajini,
> > > >
> > > > Ok, I think I got you. I wasn't calculating with the fact that the
> > parent
> > > > might not be set, therefore it could be a default user as well or
> even
> > > the
> > > > default client if nothing else is set (supposing we're talking about
> > the
> > > >  example). So if I'm correct, the quota will be applied
> > in
> > > > the order of the above points. In this case your argument is
> absolutely
> > > > valid. I'll modify the QuotaSource.
> > > >
> > > > About your last point: yes, I was hesitating a lot. I thought the
> > > interface
> > > > would be simpler but after removing the helpers it's not that scary
> > > > afterall :).
> > > > I'll start the vote.
> > > >
> > > > Viktor
> > > >
> > > >
> > > > On Thu, Jan 18, 2018 at 7:59 PM, Rajini Sivaram <
> > rajinisiva...@gmail.com
> > > >
> > > > wrote:
> > > >
> > > > > Hi Viktor,
> > > > >
> > > > > Thanks for the updates.
> > > > >
> > > > > *QuotaSource* currently has *Self/Default/Parent*. Not sure if that
> > is
> > > > > sufficient.
> > > > > For the entity , quota could be used from any of
> > these
> > > > > configs:
> > > > >
> > > > >1. /config/users//clients/
> > > > >2. /config/users//clients/
> > > > >3. /config/users/
> > > > >4. /config/users//clients/
> > > > >5. /config/users//clients/
> > > > >6. /config/users/
> > > > >7. /config/clients/
> > > > >8. /config/clients/
> > > > >
> > > > > So perhaps we should have a *QuotaSource* entry for each of these
> > > eight?
> > > > >
> > > > > A couple of minor points:
> > > > >
> > > > >- *Help Message* still uses --config.properties
> > > > >- The other AdminClient APIs don't use aliases for various
> > > > collections.
> > > > >So not sure if we need the aliases here. I think you can leave
> it
> > > > as-is
> > > > > and
> > > > >see what others think.
> > > > >
> > > > > Yes, please do start the voting thread to make it in time for the
> KIP
> > > > > freeze.
> > > > >
> > > > > Thank you,
> > > > >
> > > > > Rajini
> > > > >
> > > > >
> > > > > On Thu, Jan 18, 2018 at 6:15 PM, Viktor Somogyi <
> > > viktorsomo...@gmail.com
> > > > >
> > > > > wrote:
> > > > >
> > > > > > Rajini, I have updated the KIP as agreed. Could you please have a
> > > > second
> > > > > > look at it?
> 

[jira] [Resolved] (KAFKA-6565) Sudden unresponsiveness from broker + file descriptor leak

2018-02-19 Thread Noam Berman (JIRA)

 [ 
https://issues.apache.org/jira/browse/KAFKA-6565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Noam Berman resolved KAFKA-6565.

Resolution: Duplicate

> Sudden unresponsiveness from broker + file descriptor leak
> --
>
> Key: KAFKA-6565
> URL: https://issues.apache.org/jira/browse/KAFKA-6565
> Project: Kafka
>  Issue Type: Bug
>  Components: core
>Affects Versions: 0.10.1.0
> Environment: 3 separate clusters running 0.10.1.0 with 5 nodes  + 5 
> zookeepers.
> clients are mostly java 0.10.2.1, and a few nodejs clients 
> (https://github.com/oleksiyk/kafka). 
> throughput per broker: ~3.5MB per second. each broker is a leader of ~2500 
> partitions. 
> Debian GNU/Linux 8.8 (jessie)
> java version “1.8.0_65”
> c4.4xlarge , 1500gb gp2 disks
>Reporter: Noam Berman
>Priority: Blocker
>
> Running 3 kafka clusters v0.10.1.0  on aws, 5 brokers each (+5 zookeepers), 
> in the past week about 2-3 times a day (on all clusters, no specific brokers) 
> we encounter a situation in which a single broker stops responding to all 
> requests from clients/other brokers. at this moment all produced messages 
> fail, and other brokers start writing this to log repeatedly:
> [2018-02-15 17:56:08,949] WARN [ReplicaFetcherThread-0-878], Error in fetch 
> kafka.server.ReplicaFetcherThread$FetchRequest@772ad40c 
> (kafka.server.ReplicaFetcherThread)
>  java.io.IOException: Connection to 878 was disconnected before the response 
> was read
>  at 
> kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:115)
>  at 
> kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:112)
>  at scala.Option.foreach(Option.scala:257)
>  at 
> kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:112)
>  at 
> kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:108)
>  at 
> kafka.utils.NetworkClientBlockingOps$.recursivePoll$1(NetworkClientBlockingOps.scala:137)
>  at 
> kafka.utils.NetworkClientBlockingOps$.kafka$utils$NetworkClientBlockingOps$$pollContinuously$extension(NetworkClientBlockingOps.scala:143)
>  at 
> kafka.utils.NetworkClientBlockingOps$.blockingSendAndReceive$extension(NetworkClientBlockingOps.scala:108)
>  at 
> kafka.server.ReplicaFetcherThread.sendRequest(ReplicaFetcherThread.scala:253)
>  at kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:238)
>  at kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42)
>  at 
> kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:118)
>  at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:103)
>  at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63)
>  
> producers (examples):
>  2018-02-13 21:30:51:010 thread=[kafka-producer-network-thread | 
> catalog-index-0] WARN  Got error produce response with correlation id 8147 on 
> topic-partition catalog-1, retrying (49 attempts left). Error: 
> REQUEST_TIMED_OUT
> 2018-02-13 21:31:06:221 thread=[kafka-producer-network-thread | 
> catalog-index-0] WARN  Got error produce response with correlation id 8166 on 
> topic-partition catalog-index-18, retrying (48 attempts left). Error: 
> NOT_LEADER_FOR_PARTITION
>  
> Once this happens, file descriptors on the affected broker start increasing 
> at a high rate until it reaches its maximum.
> no relevant info logs on the affected broker, we'll try to gather debug logs 
> and attach them to the bug next time it happens.
> the big issue here is that although the broker is unresponsive, it stays in 
> the cluster and its zookeeper node isn't cleared, so there's no rebalance 
> whatsoever - all clients start failing until a manual shutdown occurs.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


Jenkins build is back to normal : kafka-1.1-jdk7 #49

2018-02-19 Thread Apache Jenkins Server
See 



Re: 1.1 release progress

2018-02-19 Thread Damian Guy
Hi,

A quick update on the 1.1 release. I'm in the process of generating RC0,
but I'm waiting to get a passing system test build before pushing. Stay
tuned!

Thanks,
Damian

On Fri, 16 Feb 2018 at 09:49 Damian Guy  wrote:

> Hi Becket,
>
> Thanks for letting me know.
>
> Damian
>
> On Fri, 16 Feb 2018 at 01:48 Ismael Juma  wrote:
>
>> Sounds good. Thanks Becket.
>>
>> Ismael
>>
>> On 15 Feb 2018 5:30 pm, "Becket Qin"  wrote:
>>
>> > Hi Ismael,
>> >
>> > Yes, I am working on the fix. Will submit patch today.
>> >
>> > Thanks,
>> >
>> > Jiangjie (Becket) Qin
>> >
>> > On Thu, Feb 15, 2018 at 2:53 PM, Ismael Juma  wrote:
>> >
>> > > Hi Becket,
>> > >
>> > > Thanks for filing that. Are you working on a fix?
>> > >
>> > > Ismael
>> > >
>> > > On Thu, Feb 15, 2018 at 2:51 PM, Becket Qin 
>> > wrote:
>> > >
>> > > > Hi Damian,
>> > > >
>> > > > I just created another ticket KAFKA-6568, which I believe should
>> also
>> > be
>> > > a
>> > > > blocker unless people disagree.
>> > > >
>> > > > Thanks,
>> > > >
>> > > > Jiangjie (Becket) Qin
>> > > >
>> > > > On Wed, Feb 14, 2018 at 8:52 AM, Damian Guy 
>> > > wrote:
>> > > >
>> > > > > Hi All,
>> > > > >
>> > > > > The first 1.1 RC is due to be cut, however we currently have 2
>> > blockers
>> > > > > outstanding:
>> > > > > https://issues.apache.org/jira/browse/KAFKA-6517 (which i suspect
>> > will
>> > > > be
>> > > > > merged shortly)
>> > > > > and
>> > > > > https://issues.apache.org/jira/browse/KAFKA-6549
>> > > > >
>> > > > > Once we have finished these issues I can create the first RC.
>> > Hopefully
>> > > > > before the end of the week.
>> > > > >
>> > > > > Regards,
>> > > > > Damian
>> > > > >
>> > > > >
>> > > > > On Mon, 5 Feb 2018 at 18:04 Damian Guy 
>> wrote:
>> > > > >
>> > > > > > Hi all,
>> > > > > >
>> > > > > > We are just over a week away from code freeze. We currently
>> have 44
>> > > > > issues
>> > > > > > in progress and 3 open blockers:
>> > > > > > https://issues.apache.org/jira/projects/KAFKA/versions/12339769
>> > > > > >
>> > > > > > If you have something that is in progress that is not going to
>> make
>> > > it
>> > > > > > into 1.1 can you please move it to a future release.
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Damian
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>


[jira] [Created] (KAFKA-6572) kafka-consumer-groups does not reset offsets to specified datetime correctly

2018-02-19 Thread Sharon Lucas (JIRA)
Sharon Lucas created KAFKA-6572:
---

 Summary: kafka-consumer-groups does not reset offsets to specified 
datetime correctly
 Key: KAFKA-6572
 URL: https://issues.apache.org/jira/browse/KAFKA-6572
 Project: Kafka
  Issue Type: Bug
Reporter: Sharon Lucas


We're seeing a problem using the kafka-consumer-groups.sh reset-offsets option 
to reset offsets to a specific date/time in our production environment.

We first tried to use the kafka-consumer-groups.sh command with the 
--reset-offsets option and with option --to-datetime 2018-02-10T00:00:00.000 in 
our staging environment and it worked correctly.  Running the following command 
changed it to start processing logs from February 12, 2018 (4 days ago) for a 
topic that had a large lag.  We did a dry run to verify before running with the 
--execute option.
{code:java}
root@ossmlpstagemon0101a:/# /opt/kafka/bin/kafka-consumer-groups.sh 
--bootstrap-server NN.NNN.NN.NN:9092 --group logstash-elasticsearch-latest 
--to-datetime 2018-02-12T00:00:00.000-06:00 --reset-offsets --topic 
staging-mon01-rg-elasticsearch --execute{code}
We stopped the kafka mirrors that process this topic before resetting the 
offsets and started the kafka mirrors after rsetting the offsets.  We verified 
that it correctly started processing logs from February 12, 2018.

Then we tried resetting offsets in a production environment for a topic that 
had a very large lag using option --to-datetime 2018-02-10T00:00:00.000 and it 
did not work as expected. We stopped the kafka mirrors that process this topic 
before resetting the offsets and did a dry run to see what the new offsets 
would be:
{code:java}
root@ossmlplon0401e:# /opt/kafka/bin/kafka-consumer-groups.sh 
--bootstrap-server NN.N.NNN.NNN:9092 --group mirror-consumer-ams03-geo-earliest 
--to-datetime 2018-02-10T00:00:00.000 --reset-offsets --topic 
prod_in-ams03-geo-ca_access
Note: This will not show information about old Zookeeper-based consumers.

^@^@^@^@
TOPIC  PARTITION  NEW-OFFSET
prod_in-ams03-geo-ca_access    52 52084147
prod_in-ams03-geo-ca_access    106    52154199
prod_in-ams03-geo-ca_access    75 52148673
prod_in-ams03-geo-ca_access    61 52130753
prod_in-ams03-geo-ca_access    49 52151667
prod_in-ams03-geo-ca_access    48 52145233
prod_in-ams03-geo-ca_access    27 52092805
prod_in-ams03-geo-ca_access    26 52139644
prod_in-ams03-geo-ca_access    65 52157504
prod_in-ams03-geo-ca_access    105    52166289
prod_in-ams03-geo-ca_access    38 52160464
prod_in-ams03-geo-ca_access    22 52093451
prod_in-ams03-geo-ca_access    4  52151660
prod_in-ams03-geo-ca_access    90 52160296
prod_in-ams03-geo-ca_access    25 52161691
prod_in-ams03-geo-ca_access    13 52145828
prod_in-ams03-geo-ca_access    56 52162867
prod_in-ams03-geo-ca_access    42 52072094
prod_in-ams03-geo-ca_access    7  52069496
prod_in-ams03-geo-ca_access    117    52087078
prod_in-ams03-geo-ca_access    32 52073732
prod_in-ams03-geo-ca_access    102    52082022
prod_in-ams03-geo-ca_access    76 52141018
prod_in-ams03-geo-ca_access    83 52154542
prod_in-ams03-geo-ca_access    72 52095051
prod_in-ams03-geo-ca_access    85 52149907
prod_in-ams03-geo-ca_access    119    52134435
prod_in-ams03-geo-ca_access    113    52159340
prod_in-ams03-geo-ca_access    55 52146597
prod_in-ams03-geo-ca_access    18 52149079
prod_in-ams03-geo-ca_access    35 52149058
prod_in-ams03-geo-ca_access    99 52143277
prod_in-ams03-geo-ca_access    41 52158872
prod_in-ams03-geo-ca_access    112    52083901
prod_in-ams03-geo-ca_access    34 52137932
prod_in-ams03-geo-ca_access    89 52158135
prod_in-ams03-geo-ca_access    40 5212
prod_in-ams03-geo-ca_access    53 52138400
prod_in-ams03-geo-ca_access    19 52144966
prod_in-ams03-geo-ca_access    44 52166404
prod_in-ams03-geo-ca_access    31 52155685
prod_in-ams03-geo-ca_access    10 52152151
prod_in-ams03-geo-ca_access    98 52145378
prod_in-ams03-geo-ca_access    69 52153436
prod_in-ams03-geo-ca_access    92 52093455
prod_in-ams03-geo-ca_access    5  52156448
prod_in-ams03-geo-ca_access    17 52063697
prod_in-ams03-geo-ca_access    24 52146410
prod_in-ams03-geo-ca_access    28 52156955
prod_in-ams03-geo-ca_access    82 52081194
prod_in-ams03-geo-ca_access    101    52153592
prod_in-ams03-geo-ca_access    58 52153765
prod_in-ams03-geo-ca_access    94 52162827
prod_in-ams03-geo-ca_access    116    52155307
prod_in-ams03-geo-ca_access    77 52072906
prod_in-ams03-geo-ca_access    68 52150283
prod_in-ams03