From dev-return-91687-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Tue Feb 13 22:43:10 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 A489F180656 for ; Tue, 13 Feb 2018 22:43:09 +0100 (CET) Received: (qmail 42177 invoked by uid 500); 13 Feb 2018 21:43:08 -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 42161 invoked by uid 99); 13 Feb 2018 21:43:07 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 13 Feb 2018 21:43:07 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 191D4C1C3A for ; Tue, 13 Feb 2018 21:43:07 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -0.021 X-Spam-Level: X-Spam-Status: No, score=-0.021 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=confluent-io.20150623.gappssmtp.com Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id BnANggngyeB8 for ; Tue, 13 Feb 2018 21:43:05 +0000 (UTC) Received: from mail-pg0-f53.google.com (mail-pg0-f53.google.com [74.125.83.53]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id A1CAC5F180 for ; Tue, 13 Feb 2018 21:43:04 +0000 (UTC) Received: by mail-pg0-f53.google.com with SMTP id j9so818966pgv.3 for ; Tue, 13 Feb 2018 13:43:04 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=confluent-io.20150623.gappssmtp.com; s=20150623; h=subject:to:references:from:organization:message-id:date:user-agent :mime-version:in-reply-to; bh=v8aawcDip7BH+iQ+vLFjQ/2eR3ztC+17ryJ/VNPtobA=; b=WJln4Lqoww+hz+MQ9P8JBn5TZx3eawkkDaYxggIjKEVKq0TFAq/IywLQcIAlhJwQJ2 gVFLd69bW+2pA1fmD8O2KhY61ew7KmV8QhbxpX44huSOv9x9MR2c76vZAs+P8yheWaP7 bAoCJ6I2m4tTMGvUOxWOX2b5eLyM2R6QmhiI1b/3JyCtq5escOlm4wFFvTiCU08klCDE wBB9zGxu+bY89zcONonzTLXf4veoejf9Zlxnj2b2plLC8ZNcmFQ/AcNuPIueY1RSMefE tEkDws12zRJQtB5H3USoGJN/aOEyHPI2H8Yn0ms2peOWigKJ2NBTnN1NlHCacu1McAfB V/Rg== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:subject:to:references:from:organization :message-id:date:user-agent:mime-version:in-reply-to; bh=v8aawcDip7BH+iQ+vLFjQ/2eR3ztC+17ryJ/VNPtobA=; b=M0KYTXFr97VqgyteXTRxeK2MdOqliIe1HJaxK21TFB2uJ0OcVBQj85P1l2OEJToTsU 5db3kGyR/dlA+RF5Mu+ewT2d2RAC9XSIKrjgmOP5SKhBVquFLqU7DzD0tlURxpuxXqDw Prr3btND3gMgiOYSnFzMC/AfPCFlHYJB942F0BjYls44+k1NZ6e+YgJk1UXAkPeTj3ro Yk2YWTYgzYeAp0BwYvbhBtjDC7XDFGjOW8/m/owSyWiMM44g9kEgRgku0sg7AIkQHS9h GUg85go1qxO/LhVVWgbqvQa+HmB2D6plABwyofDYcuF0VD2oaDQDiHlmaB41vwir21b8 kyIQ== X-Gm-Message-State: APf1xPDmMSYdbv6vJGdIDAb276dMAo45w4rZMZe0D2yveP5UwkgsZEEM H1V7vnGpxVj5S+1fZnADTz0cqvEVJrE= X-Google-Smtp-Source: AH8x227YJiYX3jQHUs0vwYKtyGLyvSPPSVIGBxVFGH/jhcLxkA45BwRUHot2fDyzTeoSzq/WE637Qw== X-Received: by 10.101.68.202 with SMTP id g10mr2134138pgs.210.1518558182479; Tue, 13 Feb 2018 13:43:02 -0800 (PST) Received: from Matthias-Sax-Macbook-Pro.local (50-0-2-20.static.sonic.net. [50.0.2.20]) by smtp.gmail.com with ESMTPSA id g63sm40154013pfg.17.2018.02.13.13.43.01 for (version=TLS1_2 cipher=ECDHE-RSA-AES128-GCM-SHA256 bits=128/128); Tue, 13 Feb 2018 13:43:01 -0800 (PST) Subject: Re: KIP-244: Add Record Header support to Kafka Streams To: dev@kafka.apache.org References: From: "Matthias J. Sax" Organization: Confluent Inc Message-ID: Date: Tue, 13 Feb 2018 13:43:01 -0800 User-Agent: Mozilla/5.0 (Macintosh; Intel Mac OS X 10.13; rv:52.0) Gecko/20100101 Thunderbird/52.6.0 MIME-Version: 1.0 In-Reply-To: Content-Type: multipart/signed; micalg=pgp-sha512; protocol="application/pgp-signature"; boundary="c5LnaZDhWlVVwRqDAydPJPJyqPUVr3odt" --c5LnaZDhWlVVwRqDAydPJPJyqPUVr3odt Content-Type: multipart/mixed; boundary="x9iZspqmXQRcHebylxbMA2JYsr8Qz7WgS"; protected-headers="v1" From: "Matthias J. Sax" To: dev@kafka.apache.org Message-ID: Subject: Re: KIP-244: Add Record Header support to Kafka Streams References: In-Reply-To: --x9iZspqmXQRcHebylxbMA2JYsr8Qz7WgS Content-Type: text/plain; charset=utf-8 Content-Language: en-US Content-Transfer-Encoding: quoted-printable Hi Jorge, I would like to unblock this KIP to make some progress. The tricky question of this work, seems to be how to expose headers at DSL level. This related to KIP-149 and KIP-159. However, for Processor API, it seems to be rather straight forward to add headers to the API. Thus, I would suggest to de-scope this KIP and add header support for Processor API only as a first step. If this is done, we can see in a second step, how to add headers at DSL level. WDYT about this proposal? If you agree, please update the JIRA and KIP accordingly. Note, that we have two JIRA that are duplicates atm. We can scope them accordingly: one for PAPI only, and second as a dependent JIRA for DSL. -Matthias On 12/30/17 3:11 PM, Jorge Esteban Quilcate Otoya wrote: > Thanks for your feedback! >=20 > 1. I was adding headers to KeyValue to support groupBy, but I think it = is > not necessary. It should be enough with mapping headers to key/value an= d > then group using current KeyValue structure. >=20 > 2. Yes. IMO key/value stores, like RocksDB, rely on KV as structure, he= nce > considering headers as part of stateful operations will not fit in this= > approach and increase complexity (I cannot think in a use-case that nee= d > this). >=20 > 3. and 4. Changes on 1. will solve this issue. >=20 > Probably I rush a bit proposing this change, I was not aware of KIP-159= or > KAFKA-5632. > If KIP-159 is adopted and we reduce this KIP to add Headers to > RecordContext will be enough, but I'm not sure about the scope of KIP-1= 59. > If it includes stateful operations will be difficult to implemented as > stated in 2. >=20 > Cheers, > Jorge. >=20 > El mar., 26 dic. 2017 a las 20:04, Matthias J. Sax () > escribi=C3=B3: >=20 >> Thanks for the KIP Jorge, >> >> As Bill pointed out already, we should be careful with adding new >> overloads as this contradicts the work done via KIP-182. >> >> This KIP also seems to be related to KIP-149 and KIP-159. Are you awar= e >> of them? Both have quite long DISCUSS threads, but it might be worth >> browsing through them. >> >> A few further questions: >> >> - why do you want to add the headers to `KeyValue`? I am not sure if = we >> should consider headers as optional metadata and add it to >> `RecordContext` similar to timestamp, offset, etc. only >=20 >=20 >> - You only include stateless single-record transformations at the DSL= >> level. Do you suggest that all other operator just drop headers on the= >> floor? >> >> - Why do you only want to put headers into in-memory and cache but no= t >> RocksDB store? What do you mean by "pass through"? IMHO, all stores >> should behave the same at DSL level. >> -> if we store the headers in the state stores, what is the upgrade= >> path? >> >> - Why do we need to store record header in state in the first place, = if >> we exclude stateful operator at DSL level? >> >> >> What is the motivation for the "border lines" you choose? >> >> >> -Matthias >> >> >> On 12/21/17 8:18 AM, Bill Bejeck wrote: >>> Jorge, >>> >>> Thanks for the KIP, I know this is a feature others in the community = have >>> been interested in getting into Kafka Streams. >>> >>> I took a quick pass over it, and I have one initial question. >>> >>> We recently reduced overloads with KIP-182, and in this KIP we are >>> increasing them again. >>> >>> I can see from the KIP why they are necessary, but I'm wondering if t= here >>> is something else we can do to cut down on the overloads introduced. = I >>> don't have any sound suggestions ATM, so I'll have to think about it = some >>> more, but I wanted to put the thought out there. >>> >>> Thanks, >>> Bill >>> >>> On Thu, Dec 21, 2017 at 9:06 AM, Jorge Esteban Quilcate Otoya < >>> quilcate.jorge@gmail.com> wrote: >>> >>>> Hi all, >>>> >>>> I have created a KIP to add Record Headers support to Kafka Streams = API: >>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP- >>>> 244%3A+Add+Record+Header+support+to+Kafka+Streams >>>> >>>> >>>> The main goal is to be able to use headers to filter, map and proces= s >>>> records as streams. Stateful processing (joins, windows) are not >>>> considered. >>>> >>>> Proposed changes/Draft: >>>> https://github.com/apache/kafka/compare/trunk...jeqo:streams-headers= >>>> >>>> Feedback and suggestions are more than welcome. >>>> >>>> Cheers, >>>> >>>> Jorge. >>>> >>> >> >> >=20 --x9iZspqmXQRcHebylxbMA2JYsr8Qz7WgS-- --c5LnaZDhWlVVwRqDAydPJPJyqPUVr3odt Content-Type: application/pgp-signature; name="signature.asc" Content-Description: OpenPGP digital signature Content-Disposition: attachment; filename="signature.asc" -----BEGIN PGP SIGNATURE----- Comment: GPGTools - https://gpgtools.org iQJKBAEBCgA0FiEEFpAnjJ4fbvgzKNUmjQjbao0qTDYFAlqDW+UWHG1hdHRoaWFz QGNvbmZsdWVudC5pbwAKCRCNCNtqjSpMNk0FD/9zI6wTVtVlKc8K/nGUgCFy398X QzLjYNRwYcaLDp+4+S8d+aSpHWOiwKDPgGFiVD95v/IoQimYwHzsom7hSZLz+4kM qg3UT6EAnwm3H/tGKtf0yzuCanJd+JDNrBvST5kfwMRzbvfudKfpXAQN/RRUZr+7 /o5tMnvU8EgKH2mOY2l2ozkFa8ER1gNbuLrQn7iJ7HwbV5tCZLGCqzLRBXq/Y5jc uwEmZjl2MtqnvYPutu4m4DZ/EbW4DgtcMtc/SBE1FyF+hSiNEIs/uRVYnOpAYsSb TEIGpwNqSKEweag0Ianmeh2K7hbZP1KW5v9zO24Ml4yXHNb8omzqdnZco3WpNz9L zpc1uMy0loMa29pBuMiATtOkgRlGvmsgMcIjsFkXEIM5dzC6uhPkl2ZU298Kuo9y xhwOXXFbMzdIlOsHw4hagacHJdKIeoR5YC+NasipDKX2zUoxuuu+gcJwPh779XTR PZNns5rdwWq4533MzGMfx7H+5o/+QCtIT3snGaBsSGhNCuK0c58VvvA2o4QWh0Uw dPEEHbxJo3eFM/zP1CaBVX6nRQKdvckW8YV5lW+qsRsyBRMk7x/bCBsbyhhG0KcA Dm1l/UQ2KaXmZQVyQE7HN9xBdFsMxyn2CDsw0FfJbOm5+1EEW3hl5x6ygzadxkTZ OaUDx7p3sf2YRCUb8A== =iLh1 -----END PGP SIGNATURE----- --c5LnaZDhWlVVwRqDAydPJPJyqPUVr3odt--