From dev-return-105148-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Thu Jun 20 20:44:01 2019 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 [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 9FAC4180670 for ; Thu, 20 Jun 2019 22:44:01 +0200 (CEST) Received: (qmail 25726 invoked by uid 500); 20 Jun 2019 20:43:59 -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 25709 invoked by uid 99); 20 Jun 2019 20:43:58 -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; Thu, 20 Jun 2019 20:43:58 +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 19E04180C3E for ; Thu, 20 Jun 2019 20:43:58 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.8 X-Spam-Level: * X-Spam-Status: No, score=1.8 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, DKIM_VALID_EF=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H2=-0.001, SPF_HELO_NONE=0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd3-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=gmail.com Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id jt2SnYtGJR2G for ; Thu, 20 Jun 2019 20:43:55 +0000 (UTC) Received: from mail-ot1-f45.google.com (mail-ot1-f45.google.com [209.85.210.45]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id AB0695FB37 for ; Thu, 20 Jun 2019 20:43:54 +0000 (UTC) Received: by mail-ot1-f45.google.com with SMTP id l15so4073398otn.9 for ; Thu, 20 Jun 2019 13:43:54 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20161025; h=mime-version:references:in-reply-to:from:date:message-id:subject:to; bh=zBqvP99ZaeLVj7fYrFlJIwi8HnCExBH6hVAG0eS9a3s=; b=hFiD+19ID/KpCp7BYUdEz/kExFzNZyT4gxbvd2pyPwOkI5n0pzDeixLjtw8yF33BFs ceyP5pD54lQJk65GYO6+RPCsFNz+Ss9rIELl901nSMAdHA62ln8oGQhpdbI/E3ojl4P8 2z4Y4RoEEJfJqIS0/00ZE8G8fj0CbT2qe927/9xviW8Gy+PpGZ/DVPWSyqqovDdW12+Q w8eAj21244uEhQ4onnvv0zJo1OrnLA9IM1LlwNnfZzkJ5Sgrm9dzC9BmYMzZ9VGJOqah Q9/F3crs1EWDaMSfisxWGOWq9DhNyHXzKMgV7fd29qw0ysqIE0PqWvaxio4O3Dx+2rzN 2HRg== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:references:in-reply-to:from:date :message-id:subject:to; bh=zBqvP99ZaeLVj7fYrFlJIwi8HnCExBH6hVAG0eS9a3s=; b=YfL6gpG71CGLLZmYPXzz5TjddVfHg6qLTxFV9wkFXJYgFmBTpCWKGzt+1tne84V6E6 cQRKyYH4YnoMF2Lzpk5tJsUSi3Oh+K4FMg4XJ8RPktCh7DfRy5o0Th8yOGGnwNFxKV+A 4XGkr3LIydARpzIQTEAUSkAD9JRiGOJfmedCEy0p7x36W3sMJPJhG7Lx+yBiGQPBsInp 8YkbPEdwzyl59JuTHFeZ/tdtMIRRV8lJVcrCLX6xWrScbgMJfaFJdvDkm5v/x7HdINzV YluAGlxwTw8+VFJBG1eVKnmvQs+H9cUZLRNzh/rk/Qq6JU+oUNSL7sEn8Mvg8HsD+XB2 YBaQ== X-Gm-Message-State: APjAAAVN/WPzA7yPCnvFOpSRCIvaEE8YZbNoi0nffz4VJdp0tdiy//8a N14pdLk7qf+zDwWOAkpiGn41c4q9Hjfzq2miuDtzf04jd4c= X-Google-Smtp-Source: APXvYqz2pCFO82OeJE3lCOwx+HBShK2L1UlthP5BD1enaXbHKKp6kFCm69V+jsbKlro5XqItM89VDPUbWfzw+bFtEoo= X-Received: by 2002:a9d:5616:: with SMTP id e22mr29450225oti.218.1561063432872; Thu, 20 Jun 2019 13:43:52 -0700 (PDT) MIME-Version: 1.0 References: In-Reply-To: From: Guozhang Wang Date: Thu, 20 Jun 2019 13:43:41 -0700 Message-ID: Subject: Re: [DISCUSS] KIP-478 Strongly Typed Processor API To: dev Content-Type: multipart/alternative; boundary="000000000000636853058bc767b1" --000000000000636853058bc767b1 Content-Type: text/plain; charset="UTF-8" Hi John, Thanks for KIP! I've a few comments below: 1. So far the "Motivation" section is very general, and the only concrete example that I have in mind is `TransformValues#punctuate`. Do we have any other concrete issues that drive this KIP? If not then I feel better to narrow the scope of this KIP to: 1.a) modifying ProcessorContext only with the output types on forward. 1.b) modifying Transformer signature to have generics of ProcessorContext, and then lift the restricting of not using punctuate: if user did not follow the enforced typing and just code without generics, they will get warning at compile time and get run-time error if they forward wrong-typed records, which I think would be acceptable. I feel this would be a good solution for this specific issue; again, feel free to update the wiki page with other known issues that cannot be resolved. 2. If, we want to go with the current scope then my next question would be, how much breakage we would introducing if we just modify the Processor signature directly? My feeling is that DSL users would be most likely not affected and PAPI users only need to modify a few lines on class declaration. I feel it worth doing some research on this part and then decide if we really want to bite the bullet of duplicated Processor / ProcessorSupplier classes for maintaining compatibility. Guozhang On Wed, Jun 19, 2019 at 12:21 PM John Roesler wrote: > Hi all, > > In response to the feedback so far, I changed the package name from > `processor2` to `processor.generic`. > > Thanks, > -John > > On Mon, Jun 17, 2019 at 4:49 PM John Roesler wrote: > > > > Thanks for the feedback, Sophie! > > > > I actually felt a little uneasy when I wrote that remark, because it's > > not restricted at all in the API, it's just available to you if you > > choose to give your stores and context the same parameters. So, I > > think your use case is valid, and also perfectly permissable under the > > current KIP. Sorry for sowing confusion on my own discussion thread! > > > > I'm not crazy about the package name, either. I went with it only > > because there's seemingly nothing special about the new package except > > that it can't have the same name as the old one. Otherwise, the > > existing "processor" and "Processor" names for the package and class > > are perfectly satisfying. Rather than pile on additional semantics, it > > seemed cleaner to just add a number to the package name. > > > > This wouldn't be the first project to do something like this... Apache > > Commons, for example, has added a "2" to the end of some of their > > packages for exactly the same reason. > > > > I'm open to any suggestions. For example, we could do something like > > org.apache.kafka.streams.typedprocessor.Processor or > > org.apache.kafka.streams.processor.typed.Processor , which would have > > just about the same effect. One microscopic thought is that, if > > there's another interface in the "processor" package that we wish to > > do the same thing to, would _could_ pile it in to "processor2", but we > > couldn't do the same if we use a package that has "typed" in the name, > > unless that change is _also_ related to types in some way. But this > > seems like a very minor concern. > > > > What's your preference? > > -John > > > > On Mon, Jun 17, 2019 at 3:56 PM Sophie Blee-Goldman > wrote: > > > > > > Hey John, thanks for writing this up! I like the proposal but there's > one > > > point that I think may be too restrictive: > > > > > > "A processor that happens to use a typed store is actually emitting the > > > same types that it is storing." > > > > > > I can imagine someone could want to leverage this new type safety > without > > > also limiting how they can interact with/use their store. As an > (admittedly > > > contrived) example, say you have an input stream of purchases of a > certain > > > type (entertainment, food, etc), and on seeing a new record you want to > > > output how many types of purchase a shopper has made more than 5 > purchases > > > of in the last month. Your state store will probably be holding some > more > > > complicated PurchaseHistory object (keyed by user), but your output is > just > > > a > > > > > > I'm also not crazy about "processor2" as the package name ... not sure > what > > > a better one would be though (something with "typed"?) > > > > > > On Mon, Jun 17, 2019 at 12:47 PM John Roesler > wrote: > > > > > > > Hi all, > > > > > > > > I'd like to propose KIP-478 ( > https://cwiki.apache.org/confluence/x/2SkLBw > > > > ). > > > > > > > > This proposal would add output type bounds to the Processor interface > > > > in Kafka Streams, which enables static checking of a number of useful > > > > properties: > > > > * A processor B that consumes the output of processor A is actually > > > > expecting the same types that processor A produces. > > > > * A processor that happens to use a typed store is actually emitting > > > > the same types that it is storing. > > > > * A processor is simply forwarding the expected types in all code > paths. > > > > * Processors added via the Streams DSL, which are not permitted to > > > > forward results at all are statically prevented from doing so by the > > > > compiler > > > > > > > > Internally, we can use the above properties to achieve a much higher > > > > level of confidence in the Streams DSL implementation's correctness. > > > > Actually, while doing the POC, I found a few bugs and mistakes, which > > > > become structurally impossible with KIP-478. > > > > > > > > Additionally, the stronger types dramatically improve the > > > > self-documentation of our Streams internal implementations, which > > > > makes it much easier for new contributors to ramp up with confidence. > > > > > > > > Thanks so much for your consideration! > > > > -John > > > > > -- -- Guozhang --000000000000636853058bc767b1--