From dev-return-99195-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Wed Oct 17 17:34:22 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 DA20318061A for ; Wed, 17 Oct 2018 17:34:21 +0200 (CEST) Received: (qmail 23969 invoked by uid 500); 17 Oct 2018 15:34:20 -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 23957 invoked by uid 99); 17 Oct 2018 15:34:20 -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; Wed, 17 Oct 2018 15:34:19 +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 9F88AC86EE for ; Wed, 17 Oct 2018 15:34:19 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 2.398 X-Spam-Level: ** X-Spam-Status: No, score=2.398 tagged_above=-999 required=6.31 tests=[DKIMWL_WL_MED=-0.001, DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, HTML_MESSAGE=2, KAM_NUMSUBJECT=0.5, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H2=-0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd1-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 (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id qP1Ec51QfTQJ for ; Wed, 17 Oct 2018 15:34:18 +0000 (UTC) Received: from mail-oi1-f172.google.com (mail-oi1-f172.google.com [209.85.167.172]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTPS id 54A145F382 for ; Wed, 17 Oct 2018 15:34:18 +0000 (UTC) Received: by mail-oi1-f172.google.com with SMTP id v69-v6so21446350oif.1 for ; Wed, 17 Oct 2018 08:34:18 -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=jLBiV26kW2jllzFjjEilO70kAgXFkGKG2ZuskX+FNQ8=; b=gB0D7Rekt6IRDq/9bqxi78DlKpqlSYBE0XNsWPBB21l7eWWtE6uMTUqhhI+MwQ97Ib s8kpcEZLawS2Rrf3qDUn80d0xz4N79KGPeDFrS8aaRNAZhVQdR1a+MBxy+o+VetPB4BT EiH8Qta6S37EQFR0wXuwP2kkzWcPQfJSOjJenTrNqCtQT6oBPdxdrdpgxFWEJ3n51GxC yvtlXlAY7OHDgaXsApGKdylOuiaC9sbjqkL/vsxWTtbM+FBMeWx9fDp9eMEOK3c68Ilu ucB/Rc5SskLuzUaaaM2UzTgz+BMqyGA36ADPNcWn3Mqqt4SUW0wk8GTW4uEdiVyhz4zB smfA== 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=jLBiV26kW2jllzFjjEilO70kAgXFkGKG2ZuskX+FNQ8=; b=oG2N2l9NK/jBr3wD69+v9LzioMot5QhTkxaedNx7FR4XqzmhpNf+3HIZMbvHgr+RJI zHNkS5OpltIPkSBzwMf63w0uWLnSQ5eksY1SG5qxcSq4srHxDc1aajhOpomt++Ari4i3 veDDtFIAIR0Jfao6YAkrT0j08T2+3vBAhDavhg7AZrh0nWaKYnm3dVS8FpfHi21Tc62y Ga8eg52aQaHUAQ9udvRLYG5mXmB0ScklDQNkJTgohkyPID6JUfFa33gZDY/wrGTfpDwu 6N+i8C6j2HlYym/0DuWzVrq5ES5cX5grjRgbxuTmeumHaLjFz5141z6UikoOeB5scpDi Po2g== X-Gm-Message-State: ABuFfojFVGcrQWiSDCjX0PoiAAh31QawHq2pT7Seq6wwGZdrtF6uwExO MnfWQ6T3P/R+5RsE4A8h4AwxbBcBRxCHgsLdwqAhZQ== X-Google-Smtp-Source: ACcGV63u+sqlYGi42xu8R2WWtfsB0wEYWs+6mKVd94aabgNxI+uoO8bBsVA3V+/G6X1yvCLadEBy+rNqrFOqZHPij+c= X-Received: by 2002:aca:c42:: with SMTP id i2-v6mr2181774oiy.256.1539790456999; Wed, 17 Oct 2018 08:34:16 -0700 (PDT) MIME-Version: 1.0 References: <5BC5D244.8000202@trivago.com> <5BC72664.4000208@trivago.com> In-Reply-To: <5BC72664.4000208@trivago.com> From: Ryanne Dolan Date: Wed, 17 Oct 2018 10:34:03 -0500 Message-ID: Subject: Re: [DISCUSS] KIP-382: MirrorMaker 2.0 To: dev@kafka.apache.org, "Jan.Filipiak@trivago.com" Content-Type: multipart/alternative; boundary="00000000000037d58505786e67ce" --00000000000037d58505786e67ce Content-Type: text/plain; charset="UTF-8" Jan, these are two separate issues. 1) consumer coordination should not, ideally, involve unreliable or slow connections. Naively, a KafkaSourceConnector would coordinate via the source cluster. We can do better than this, but I'm deferring this optimization for now. 2) exactly-once between two clusters is mind-bending. But keep in mind that transactions are managed by the producer, not the consumer. In fact, it's the producer that requests that offsets be committed for the current transaction. Obviously, these offsets are committed in whatever cluster the producer is sending to. These two issues are closely related. They are both resolved by not coordinating or committing via the source cluster. And in fact, this is the general model of SourceConnectors anyway, since most SourceConnectors _only_ have a destination cluster. If there is a lot of interest here, I can expound further on this aspect of MM2, but again I think this is premature until this first KIP is approved. I intend to address each of these in separate KIPs following this one. Ryanne On Wed, Oct 17, 2018 at 7:09 AM Jan Filipiak wrote: > This is not a performance optimisation. Its a fundamental design choice. > > > I never really took a look how streams does exactly once. (its a trap > anyways and you usually can deal with at least once donwstream pretty > easy). But I am very certain its not gonna get somewhere if offset > commit and record produce cluster are not the same. > > Pretty sure without this _design choice_ you can skip on that exactly > once already > > Best Jan > > On 16.10.2018 18:16, Ryanne Dolan wrote: > > > But one big obstacle in this was > > always that group coordination happened on the source cluster. > > > > Jan, thank you for bringing up this issue with legacy MirrorMaker. I > > totally agree with you. This is one of several problems with MirrorMaker > > I intend to solve in MM2, and I already have a design and prototype that > > solves this and related issues. But as you pointed out, this KIP is > > already rather complex, and I want to focus on the core feature set > > rather than performance optimizations for now. If we can agree on what > > MM2 looks like, it will be very easy to agree to improve its performance > > and reliability. > > > > That said, I look forward to your support on a subsequent KIP that > > addresses consumer coordination and rebalance issues. Stay tuned! > > > > Ryanne > > > > On Tue, Oct 16, 2018 at 6:58 AM Jan Filipiak > > wrote: > > > > Hi, > > > > Currently MirrorMaker is usually run collocated with the target > > cluster. > > This is all nice and good. But one big obstacle in this was > > always that group coordination happened on the source cluster. So > when > > then network was congested, you sometimes loose group membership and > > have to rebalance and all this. > > > > So one big request from we would be the support of having > coordination > > cluster != source cluster. > > > > I would generally say a LAN is better than a WAN for doing group > > coordinaton and there is no reason we couldn't have a group consuming > > topics from a different cluster and committing offsets to another > > one right? > > > > Other than that. It feels like the KIP has too much features where > many > > of them are not really wanted and counter productive but I will just > > wait and see how the discussion goes. > > > > Best Jan > > > > > > On 15.10.2018 18:16, Ryanne Dolan wrote: > > > Hey y'all! > > > > > > Please take a look at KIP-382: > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0 > > > > > > Thanks for your feedback and support. > > > > > > Ryanne > > > > > > --00000000000037d58505786e67ce--