From dev-return-97146-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Tue Aug 14 23:22:30 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 06F4A180654 for ; Tue, 14 Aug 2018 23:22:29 +0200 (CEST) Received: (qmail 33812 invoked by uid 500); 14 Aug 2018 21:22:28 -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 33800 invoked by uid 99); 14 Aug 2018 21:22:28 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 14 Aug 2018 21:22:28 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id A614918019B for ; Tue, 14 Aug 2018 21:22:27 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 2.869 X-Spam-Level: ** X-Spam-Status: No, score=2.869 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, FREEMAIL_REPLY=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, T_DKIMWL_WL_MED=-0.01] autolearn=disabled Authentication-Results: spamd3-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=gmail.com Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id d1eFkUrq6DDw for ; Tue, 14 Aug 2018 21:22:24 +0000 (UTC) Received: from mail-oi0-f52.google.com (mail-oi0-f52.google.com [209.85.218.52]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTPS id 63CB85F300 for ; Tue, 14 Aug 2018 21:22:24 +0000 (UTC) Received: by mail-oi0-f52.google.com with SMTP id d189-v6so36342668oib.6 for ; Tue, 14 Aug 2018 14:22:24 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20161025; h=mime-version:in-reply-to:references:from:date:message-id:subject:to; bh=rcWcz0Yz0s7I191uubaIHW9ooFwxbFykbFfUSRFWBPw=; b=jRwptVO/gOIwvEMKqi/GQRFAkyEkxgjz5vi29qlTk/zfcxScr2mGAKcIjqSBfIfxUa /PJ6KPxxMy+jAJ9iYA6gT2niMit+WaeONBOprOp0M7Bd/wu7ZHx3r665cHpDNv4RrZcH YyVnCHd/4Dm///yQIC+x1xdYZSHtd5SvWUoaAe254oTD5tmhi8+3IEROy85jCA/KCdCF +JoU5m80a9MIL4c41Qgk3EDAZlzF0LZ6t+4xaktvXrupVnEiRZsP+SZWxF0qycP3mO7q Ei13j9rsOu73nw4FnTxdHe2w/8I8MSTQJf5T2N2w4K4vK0neb8Uiou7640zbhZtOfj0D 01bg== 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=rcWcz0Yz0s7I191uubaIHW9ooFwxbFykbFfUSRFWBPw=; b=bQatmsRqT6FJ4vEhbcH2V7C4clDdhCMmRo+sYZbxnGsa79kAQuPxKJKn1dwdY7lZBo 4tEz5ROr+wX0+fORnA6jXVwJMHuWhj71kBtRsZ3Rap4xNQUi/UVFISsYd92Nvxio6GzY 1QBY7nx6+0TJ8jejjalMCxVAcI6ubt5OFN8iquE+Gf8KkYVzN+gUdGxjAlGuxT9U4COo xzxpw8/5gm1z8WCurKCB2YIAihIoe4LJ38ME/SqxPh+jXeolq7fe9DlGr9aQC2u8ixdt 3Kqbgh6W7ep/jfq0qfj6/fvr5tLt1sWuu8RK/948B/+SFxl+hwA7VwkGHcXhGfcvmHz5 DUog== X-Gm-Message-State: AOUpUlGztkxRZOLKOyFUhGzsVb9pqoAPSnxJborzI8xKgOXm2bR84DTd f1qHhSo84kWgeL6Tt09vuSV0+DUznxbc60b9e5cUyX+i4Tw= X-Google-Smtp-Source: AA+uWPwjOVpI+YQmnBw+vuHHlgvqgmoXj1cRa6/VNtXDa8JNWbPxg+ljj+txt2qcXiqAEIJ4F5/xGNiHYB2ojAgLdSo= X-Received: by 2002:aca:b4d4:: with SMTP id d203-v6mr25992183oif.79.1534281743261; Tue, 14 Aug 2018 14:22:23 -0700 (PDT) MIME-Version: 1.0 Received: by 2002:a4a:4355:0:0:0:0:0 with HTTP; Tue, 14 Aug 2018 14:22:22 -0700 (PDT) In-Reply-To: <5B727921.9070403@trivago.com> References: <5B7134A0.1020000@trivago.com> <5B727921.9070403@trivago.com> From: Guozhang Wang Date: Tue, 14 Aug 2018 14:22:22 -0700 Message-ID: Subject: Re: KIP-213 - Scalable/Usable Foreign-Key KTable joins - Rebooted. To: dev , Bill Bejeck Content-Type: multipart/alternative; boundary="0000000000004ae7d405736bce48" --0000000000004ae7d405736bce48 Content-Type: text/plain; charset="UTF-8" Hello Adam, As for your question regarding GraphNodes, it is for extending Streams optimization framework. You can find more details on https://issues.apache.org/jira/browse/KAFKA-6761. The main idea is that instead of directly building up the "physical topology" (represented as Topology in the public package, and internally built as the ProcessorTopology class) while users are specifying the transformation operators, we first keep it as a "logical topology" (represented as GraphNode inside InternalStreamsBuilder). And then only execute the optimization and the construction of the "physical" Topology when StreamsBuilder.build() is called. Back to your question, I think it makes more sense to add a new type of StreamsGraphNode (maybe you can consider inheriting from the BaseJoinProcessorNode). Note that although in the Topology we will have multiple connected ProcessorNodes to represent a (foreign-key) join, we still want to keep it as a single StreamsGraphNode, or just a couple of them in the logical representation so that in the future we can construct the physical topology differently (e.g. having another way than the current distributed hash-join). ------------------------------------------------------- Back to your questions to KIP-213, I think Jan has summarized it pretty-well. Note that back then we do not have headers support so we have to do such "key-widening" approach to ensure ordering. Guozhang On Mon, Aug 13, 2018 at 11:39 PM, Jan Filipiak wrote: > Hi Adam, > > I love how you are on to this already! I resolve this by "key-widening" I > treat the result of FKA,and FKB differently. > As you can see the output of my join has a Combined Key and therefore I > can resolve the "race condition" in a group by > if I so desire. > > I think this reflects more what happens under the hood and makes it more > clear to the user what is going on. The Idea > of hiding this behind metadata and handle it in the DSL is from my POV > unideal. > > To write into your example: > > key + A, null) > (key +B, ) > > is what my output would look like. > > > Hope that makes sense :D > > Best Jan > > > > > > On 13.08.2018 18:16, Adam Bellemare wrote: > >> Hi Jan >> >> If you do not use headers or other metadata, how do you ensure that >> changes >> to the foreign-key value are not resolved out-of-order? >> ie: If an event has FK = A, but you change it to FK = B, you need to >> propagate both a delete (FK=A -> null) and an addition (FK=B). In my >> solution, without maintaining any metadata, it is possible for the final >> output to be in either order - the correctly updated joined value, or the >> null for the delete. >> >> (key, null) >> (key, ) >> >> or >> >> (key, ) >> (key, null) >> >> I looked back through your code and through the discussion threads, and >> didn't see any information on how you resolved this. I have a version of >> my >> code working for 2.0, I am just adding more integration tests and will >> update the KIP accordingly. Any insight you could provide on resolving >> out-of-order semantics without metadata would be helpful. >> >> Thanks >> Adam >> >> >> On Mon, Aug 13, 2018 at 3:34 AM, Jan Filipiak >> wrote: >> >> Hi, >>> >>> Happy to see that you want to make an effort here. >>> >>> Regarding the ProcessSuppliers I couldn't find a way to not rewrite the >>> joiners + the merger. >>> The re-partitioners can be reused in theory. I don't know if repartition >>> is optimized in 2.0 now. >>> >>> I made this >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-241+ >>> KTable+repartition+with+compacted+Topics >>> back then and we are running KIP-213 with KIP-241 in combination. >>> >>> For us it is vital as it minimized the size we had in our repartition >>> topics plus it removed the factor of 2 in events on every message. >>> I know about this new "delete once consumer has read it". I don't think >>> 241 is vital for all usecases, for ours it is. I wanted >>> to use 213 to sneak in the foundations for 241 aswell. >>> >>> I don't quite understand what a PropagationWrapper is, but I am certain >>> that you do not need RecordHeaders >>> for 213 and I would try to leave them out. They either belong to the DSL >>> or to the user, having a mixed use is >>> to be avoided. We run the join with 0.8 logformat and I don't think one >>> needs more. >>> >>> This KIP will be very valuable for the streams project! I couldn't never >>> convince myself to invest into the 1.0+ DSL >>> as I used almost all my energy to fight against it. Maybe this can also >>> help me see the good sides a little bit more. >>> >>> If there is anything unclear with all the text that has been written, >>> feel >>> free to just directly cc me so I don't miss it on >>> the mailing list. >>> >>> Best Jan >>> >>> >>> >>> >>> >>> On 08.08.2018 15:26, Adam Bellemare wrote: >>> >>> More followup, and +dev as Guozhang replied to me directly previously. >>>> >>>> I am currently porting the code over to trunk. One of the major changes >>>> since 1.0 is the usage of GraphNodes. I have a question about this: >>>> >>>> For a foreignKey joiner, should it have its own dedicated node type? Or >>>> would it be advisable to construct it from existing GraphNode >>>> components? >>>> For instance, I believe I could construct it from several >>>> OptimizableRepartitionNode, some SinkNode, some SourceNode, and several >>>> StatefulProcessorNode. That being said, there is some underlying >>>> complexity >>>> to each approach. >>>> >>>> I will be switching the KIP-213 to use the RecordHeaders in Kafka >>>> Streams >>>> instead of the PropagationWrapper, but conceptually it should be the >>>> same. >>>> >>>> Again, any feedback is welcomed... >>>> >>>> >>>> On Mon, Jul 30, 2018 at 9:38 AM, Adam Bellemare < >>>> adam.bellemare@gmail.com >>>> wrote: >>>> >>>> Hi Guozhang et al >>>> >>>>> I was just reading the 2.0 release notes and noticed a section on >>>>> Record >>>>> Headers. >>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP- >>>>> 244%3A+Add+Record+Header+support+to+Kafka+Streams+Processor+API >>>>> >>>>> I am not yet sure if the contents of a RecordHeader is propagated all >>>>> the >>>>> way through the Sinks and Sources, but if it is, and if it remains >>>>> attached >>>>> to the record (including null records) I may be able to ditch the >>>>> propagationWrapper for an implementation using RecordHeader. I am not >>>>> yet >>>>> sure if this is doable, so if anyone understands RecordHeader impl >>>>> better >>>>> than I, I would be happy to hear from you. >>>>> >>>>> In the meantime, let me know of any questions. I believe this PR has a >>>>> lot >>>>> of potential to solve problems for other people, as I have encountered >>>>> a >>>>> number of other companies in the wild all home-brewing their own >>>>> solutions >>>>> to come up with a method of handling relational data in streams. >>>>> >>>>> Adam >>>>> >>>>> >>>>> On Fri, Jul 27, 2018 at 1:45 AM, Guozhang Wang >>>>> wrote: >>>>> >>>>> Hello Adam, >>>>> >>>>>> Thanks for rebooting the discussion of this KIP ! Let me finish my >>>>>> pass >>>>>> on the wiki and get back to you soon. Sorry for the delays.. >>>>>> >>>>>> Guozhang >>>>>> >>>>>> On Tue, Jul 24, 2018 at 6:08 AM, Adam Bellemare < >>>>>> adam.bellemare@gmail.com >>>>>> >>>>>> wrote: >>>>>>> Let me kick this off with a few starting points that I would like to >>>>>>> generate some discussion on. >>>>>>> >>>>>>> 1) It seems to me that I will need to repartition the data twice - >>>>>>> once >>>>>>> on >>>>>>> the foreign key, and once back to the primary key. Is there anything >>>>>>> I >>>>>>> am >>>>>>> missing here? >>>>>>> >>>>>>> 2) I believe I will also need to materialize 3 state stores: the >>>>>>> prefixScan >>>>>>> SS, the highwater mark SS (for out-of-order resolution) and the final >>>>>>> state >>>>>>> store, due to the workflow I have laid out. I have not thought of a >>>>>>> better >>>>>>> way yet, but would appreciate any input on this matter. I have gone >>>>>>> back >>>>>>> through the mailing list for the previous discussions on this KIP, >>>>>>> and >>>>>>> I >>>>>>> did not see anything relating to resolving out-of-order compute. I >>>>>>> cannot >>>>>>> see a way around the current three-SS structure that I have. >>>>>>> >>>>>>> 3) Caching is disabled on the prefixScan SS, as I do not know how to >>>>>>> resolve the iterator obtained from rocksDB with that of the cache. In >>>>>>> addition, I must ensure everything is flushed before scanning. Since >>>>>>> the >>>>>>> materialized prefixScan SS is under "control" of the function, I do >>>>>>> not >>>>>>> anticipate this to be a problem. Performance throughput will need to >>>>>>> be >>>>>>> tested, but as Jan observed in his initial overview of this issue, it >>>>>>> is >>>>>>> generally a surge of output events which affect performance moreso >>>>>>> than >>>>>>> the >>>>>>> flush or prefixScan itself. >>>>>>> >>>>>>> Thoughts on any of these are greatly appreciated, since these >>>>>>> elements >>>>>>> are >>>>>>> really the cornerstone of the whole design. I can put up the code I >>>>>>> have >>>>>>> written against 1.0.2 if we so desire, but first I was hoping to just >>>>>>> tackle some of the fundamental design proposals. >>>>>>> >>>>>>> Thanks, >>>>>>> Adam >>>>>>> >>>>>>> >>>>>>> >>>>>>> On Mon, Jul 23, 2018 at 10:05 AM, Adam Bellemare < >>>>>>> adam.bellemare@gmail.com> >>>>>>> wrote: >>>>>>> >>>>>>> Here is the new discussion thread for KIP-213. I picked back up on >>>>>>> the >>>>>>> KIP >>>>>>> >>>>>>> as this is something that we too at Flipp are now running in >>>>>>>> >>>>>>>> production. >>>>>>> >>>>>>> Jan started this last year, and I know that Trivago is also using >>>>>>>> >>>>>>>> something >>>>>>> >>>>>>> similar in production, at least in terms of APIs and functionality. >>>>>>>> >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP- >>>>>>>> 213+Support+non-key+joining+in+KTable >>>>>>>> >>>>>>>> I do have an implementation of the code for Kafka 1.0.2 (our local >>>>>>>> production version) but I won't post it yet as I would like to focus >>>>>>>> >>>>>>>> on the >>>>>>> >>>>>>> workflow and design first. That being said, I also need to add some >>>>>>>> >>>>>>>> clearer >>>>>>> >>>>>>> integration tests (I did a lot of testing using a non-Kafka Streams >>>>>>>> framework) and clean up the code a bit more before putting it in a >>>>>>>> PR >>>>>>>> against trunk (I can do so later this week likely). >>>>>>>> >>>>>>>> Please take a look, >>>>>>>> >>>>>>>> Thanks >>>>>>>> >>>>>>>> Adam Bellemare >>>>>>>> >>>>>>>> >>>>>>>> >>>>>>>> -- >>>>>> -- Guozhang >>>>>> >>>>>> >>>>>> > -- -- Guozhang --0000000000004ae7d405736bce48--