From dev-return-92449-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Sat Mar 17 20:00:38 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 23AB3180652 for ; Sat, 17 Mar 2018 20:00:37 +0100 (CET) Received: (qmail 32213 invoked by uid 500); 17 Mar 2018 19:00:36 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 32197 invoked by uid 99); 17 Mar 2018 19:00:35 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 17 Mar 2018 19:00:35 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 7EF27C0115 for ; Sat, 17 Mar 2018 19:00:35 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.98 X-Spam-Level: * X-Spam-Status: No, score=1.98 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd4-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=confluent-io.20150623.gappssmtp.com Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id 4DmhRtpJbxz6 for ; Sat, 17 Mar 2018 19:00:32 +0000 (UTC) Received: from mail-ot0-f173.google.com (mail-ot0-f173.google.com [74.125.82.173]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTPS id 5F6BA5F202 for ; Sat, 17 Mar 2018 19:00:32 +0000 (UTC) Received: by mail-ot0-f173.google.com with SMTP id r30-v6so13647874otr.2 for ; Sat, 17 Mar 2018 12:00:32 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=confluent-io.20150623.gappssmtp.com; s=20150623; h=mime-version:in-reply-to:references:from:date:message-id:subject:to; bh=Zg1tBFUnlF3qFv1bIFA0WdoSE1McZ/Zg16u4TSOi4ao=; b=L8QcBxRtRuVc9DWVSowf/ugS4l7j9JJYZVYJqTxTxNJzcD/AMB3we+sM0JX6theTtC aORt8SG1PuLaW5/w+Frf4IPl9DJtBmueurKBuijgsNTdJZV6RJcYL4T7ZGEhwNqciVip anG8sR3NEevAtDwgIKtYE6ZzkdK/NpdyTIrZaixcherlvtmNE++7cVBTecIO8WTyz6np h6smYP8L4PenvaKUoKoyhcnrwErBEc3t4uYuLanhonoRJoz2gTy29OvSJJtEx6Uj/j9D A665kxXLIxZwm8Xwva7Bjz4GKgMpv7V+Tl8Pk3Aq3EiZ6CBwFbMoSGptpFfwz1glJbu7 vSRQ== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:in-reply-to:references:from:date :message-id:subject:to; bh=Zg1tBFUnlF3qFv1bIFA0WdoSE1McZ/Zg16u4TSOi4ao=; b=VMHhfS2pwYTD+QIf21BISIXrmCqy51qbBEHySUokLgNoQiTVmrjGRbJy+69GJtkHcr 4HoNtYaGqF3iTrBVlehsE0NzOPrVghlkRvCuNbz5rIS3IjSKmNJ2P3jBGjitd12SP6Eg P549vGk4KRt+QkUWxMZylr/hle3BoAWWesKvdssTQhnv3TmWbRBl2lc3/ZCbGYgoUfdv p5R2Opqa0CoH/KX0f0U7NcgpKuFdDaEktYRZXoQnded5Hg9Ouqcl30YFsqxysIkleGSr ML7Kx5bjO6K65GaeQA9hGZKtExACDCXu3lX7ZzlErGdMk7uXRklxrkQE7UQ3vImygsM0 KSig== X-Gm-Message-State: AElRT7G4DxxrgCevT9eDL8IUuoaFbyYfYi8GlqgZCdr0nMQteiohro0M feDGrpMy1+UZAbJCFpRlkTiD9nhcL5YN6E1sCCUsfw== X-Google-Smtp-Source: AG47ELtxzD7VOLPpY/yA1NwMZSE3cqI0z3sRrR9nlZjW8tUgWMBJ73OGPe4YOQwnbAxCc+b1dRtantuV8XwfIWisyP8= X-Received: by 2002:a9d:20a2:: with SMTP id x31-v6mr3973268ota.242.1521313226168; Sat, 17 Mar 2018 12:00:26 -0700 (PDT) MIME-Version: 1.0 Received: by 10.74.156.203 with HTTP; Sat, 17 Mar 2018 12:00:25 -0700 (PDT) In-Reply-To: References: From: Jason Gustafson Date: Sat, 17 Mar 2018 12:00:25 -0700 Message-ID: Subject: Re: [DISCUSSION] KIP-266: Add TimeoutException to KafkaConsumer#position() To: dev@kafka.apache.org Content-Type: multipart/alternative; boundary="0000000000007037af0567a05607" --0000000000007037af0567a05607 Content-Type: text/plain; charset="UTF-8" Hi Richard, Thanks for the updates. I'm really glad you picked this up. A couple minor comments: 1. Can you list the full set of new APIs explicitly in the KIP? Currently I only see the javadoc for `position()`. 2. We should consider adding `TimeUnit` to the new methods to avoid unit confusion. I know it's inconsistent with the poll() API, but I think it was probably a mistake not to include it there, so better not to double down on that mistake. And note that we do already have `close(long, TimeUnit)`. Other than that, I think the current KIP seems reasonable. Thanks, Jason On Wed, Mar 14, 2018 at 5:00 PM, Richard Yu wrote: > Note to all: I have included bounding commitSync() and committed() in this > KIP. > > On Sun, Mar 11, 2018 at 5:05 PM, Richard Yu > wrote: > > > Hi all, > > > > I updated the KIP where overloading position() is now the favored > approach. > > Bounding position() using requestTimeoutMs has been listed as rejected. > > > > Any thoughts? > > > > On Tue, Mar 6, 2018 at 6:00 PM, Guozhang Wang > wrote: > > > >> I agree that adding the overloads is most flexible. But going for that > >> direction we'd do that for all the blocking call that I've listed above, > >> with this timeout value covering the end-to-end waiting time. > >> > >> > >> Guozhang > >> > >> On Tue, Mar 6, 2018 at 10:02 AM, Ted Yu wrote: > >> > >> > bq. The most flexible option is to add overloads to the consumer > >> > > >> > This option is flexible. > >> > > >> > Looking at the tail of SPARK-18057, Spark dev voiced the same choice. > >> > > >> > +1 for adding overload with timeout parameter. > >> > > >> > Cheers > >> > > >> > On Mon, Mar 5, 2018 at 2:42 PM, Jason Gustafson > >> > wrote: > >> > > >> > > @Guozhang I probably have suggested all options at some point or > >> another, > >> > > including most recently, the current KIP! I was thinking that > >> practically > >> > > speaking, the request timeout defines how long the user is willing > to > >> > wait > >> > > for a response. The consumer doesn't really have a complex send > >> process > >> > > like the producer for any of these APIs, so I wasn't sure how much > >> > benefit > >> > > there would be from having more granular control over timeouts (in > the > >> > end, > >> > > KIP-91 just adds a single timeout to control the whole send). That > >> said, > >> > it > >> > > might indeed be better to avoid overloading the config as you > suggest > >> > since > >> > > at least it avoids inconsistency with the producer's usage. > >> > > > >> > > The most flexible option is to add overloads to the consumer so that > >> > users > >> > > can pass the timeout directly. I'm not sure if that is more or less > >> > > annoying than a new config, but I've found config timeouts a little > >> > > constraining in practice. For example, I could imagine users wanting > >> to > >> > > wait longer for an offset commit operation than a position lookup; > if > >> the > >> > > latter isn't timely, users can just pause the partition and continue > >> > > fetching on others. If you cannot commit offsets, however, it might > be > >> > > safer for an application to wait availability of the coordinator > than > >> > > continuing. > >> > > > >> > > -Jason > >> > > > >> > > On Sun, Mar 4, 2018 at 10:14 PM, Guozhang Wang > >> > wrote: > >> > > > >> > > > Hello Richard, > >> > > > > >> > > > Thanks for the proposed KIP. I have a couple of general comments: > >> > > > > >> > > > 1. I'm not sure if piggy-backing the timeout exception on the > >> > > > existing requestTimeoutMs configured in "request.timeout.ms" is a > >> good > >> > > > idea > >> > > > since a) it is a general config that applies for all types of > >> requests, > >> > > and > >> > > > 2) using it to cover all the phases of an API call, including > >> network > >> > > round > >> > > > trip and potential metadata refresh is shown to not be a good > idea, > >> as > >> > > > illustrated in KIP-91: > >> > > > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > >> > > > 91+Provide+Intuitive+User+Timeouts+in+The+Producer > >> > > > > >> > > > In fact, I think in KAFKA-4879 which is aimed for the same issue > as > >> > > > KAFKA-6608, > >> > > > Jason has suggested we use a new config for the API. Maybe this > >> would > >> > be > >> > > a > >> > > > more intuitive manner than reusing the request.timeout.ms config. > >> > > > > >> > > > > >> > > > 2. Besides the Consumer.position() call, there are a couple of > more > >> > > > blocking calls today that could result in infinite blocking: > >> > > > Consumer.commitSync() and Consumer.committed(), should they be > >> > considered > >> > > > in this KIP as well? > >> > > > > >> > > > 3. There are a few other APIs that are today relying on > >> > > request.timeout.ms > >> > > > already for breaking the infinite blocking, namely > >> > > Consumer.partitionFor(), > >> > > > Consumer.OffsetAndTimestamp() and Consumer.listTopics(), if we are > >> > making > >> > > > the other blocking calls to be relying a new config as suggested > in > >> 1) > >> > > > above, should we also change the semantics of these API functions > >> for > >> > > > consistency? > >> > > > > >> > > > > >> > > > Guozhang > >> > > > > >> > > > > >> > > > > >> > > > > >> > > > On Sun, Mar 4, 2018 at 11:13 AM, Richard Yu < > >> > yohan.richard.yu@gmail.com> > >> > > > wrote: > >> > > > > >> > > > > Hi all, > >> > > > > > >> > > > > I would like to discuss a potential change which would be made > to > >> > > > > KafkaConsumer: > >> > > > > https://cwiki.apache.org/confluence/pages/viewpage. > >> > > > action?pageId=75974886 > >> > > > > > >> > > > > Thanks, > >> > > > > Richard Yu > >> > > > > > >> > > > > >> > > > > >> > > > > >> > > > -- > >> > > > -- Guozhang > >> > > > > >> > > > >> > > >> > >> > >> > >> -- > >> -- Guozhang > >> > > > > > --0000000000007037af0567a05607--