Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 296C0200CF8 for ; Thu, 14 Sep 2017 20:03:25 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 2277D1609CD; Thu, 14 Sep 2017 18:03:25 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 418FF1609C6 for ; Thu, 14 Sep 2017 20:03:24 +0200 (CEST) Received: (qmail 44866 invoked by uid 500); 14 Sep 2017 18:03:18 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list user@flink.apache.org Received: (qmail 44856 invoked by uid 99); 14 Sep 2017 18:03:18 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 14 Sep 2017 18:03:18 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id D66971A67D7 for ; Thu, 14 Sep 2017 18:03:17 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.599 X-Spam-Level: X-Spam-Status: No, score=0.599 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_H2=-2.8, RCVD_IN_SORBS_SPAM=0.5, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd2-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 (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id gaVuP9wERdtQ for ; Thu, 14 Sep 2017 18:03:16 +0000 (UTC) Received: from mail-io0-f182.google.com (mail-io0-f182.google.com [209.85.223.182]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id AFFCF60D26 for ; Thu, 14 Sep 2017 18:03:14 +0000 (UTC) Received: by mail-io0-f182.google.com with SMTP id k101so2620751iod.0 for ; Thu, 14 Sep 2017 11:03:14 -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 :cc; bh=CAXCXOQAXx1G5jAPruLCU4eE5hPPpqJ9QS4ZM+Qh8Lk=; b=ixuhQ3N8Zq5pKTeGWx4U2jvyfYO8IQ5hxT8v33dzOcANcWnEI2RLZtMHgpi241Z82r JYQmmXKhDb/+WOZ6BMnCfmOuomcsePJWKsute2VDXNQFU3tlbihZwgtpbxphKdDTBR0b /CnvKiJ6hUD9Aju/0BiaEjtF97BAFpveuetFwZTD9Lvz7tQ1J0Hg7ZxPIYMW2F1FIR8t EFIKSDMqQ/6kAHsb5onYh8GYkvG6NLzgxRvo9Diy0xMureeP5XkhsuNAm4L1oMCnMISf 3VvIuwFZWL4WddGJisnoc7QMYhsToqo4VLJmyM0e1qZXZPHM9ecIzu9pPhDQ6h44zG1p OYrA== 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:cc; bh=CAXCXOQAXx1G5jAPruLCU4eE5hPPpqJ9QS4ZM+Qh8Lk=; b=mcoOvHer1WgpUD815FjCAxnVN3ZIcFeSpC8SK7oL5NcryXWV/0WvJfp0FWIi28S1hs b1oIIJM2q/ewLNoFnRioCOf/9oqxD6VX+KEYJ7h05GH2REVgS9XI+ftnpeQe3R2ezW0x qYgCl9zAqF8ibfnkfUE9h/L8W5VeVvv9QIqrMKOqYRjSFJxs7uKypaM9S7VIR1lPa62h hTJmG6KTUXBjsAwV3btEi+R0pDg0J+0t4jJXRSAw2bTJKedej61vleEMtsTdFyAKf6/j qh+zlN3D7w1qf8G3m/qqgPonFJhbzRpl591y5iphWDPNYXBngxlPUslAT1wx21b8/JNC TtJQ== X-Gm-Message-State: AHPjjUiuLmPF4K9MvAwYw5mq1jztgsocskmcd7JleIb+Dahvcr8a4x6K t1Q8T4ny/mz6T+EvHZoDYLnPh0Fj1mijn70n7tM= X-Google-Smtp-Source: AOwi7QDMT7T04HUmiW6cWY69YYYXVIWd6/WocbxD77K/qYkNJfN5Yd3UbbVut4CZavwVmtecO5sC9tq9/SZ/1GV2GQQ= X-Received: by 10.202.82.19 with SMTP id g19mr20895414oib.99.1505412193376; Thu, 14 Sep 2017 11:03:13 -0700 (PDT) MIME-Version: 1.0 Received: by 10.74.136.208 with HTTP; Thu, 14 Sep 2017 11:03:12 -0700 (PDT) In-Reply-To: References: From: Eron Wright Date: Thu, 14 Sep 2017 11:03:12 -0700 Message-ID: Subject: Re: Flink flick cancel vs stop To: Ufuk Celebi Cc: Elias Levy , user , Till Rohrmann Content-Type: multipart/alternative; boundary="001a113d840006df3d05592a1750" archived-at: Thu, 14 Sep 2017 18:03:25 -0000 --001a113d840006df3d05592a1750 Content-Type: text/plain; charset="UTF-8" Content-Transfer-Encoding: quoted-printable I too am curious about stop vs cancel. I'm trying to understand the motivations a bit more. The current behavior of stop is basically that the sources become bounded, leading to the job winding down. The interesting question is how best to support 'planned' maintenance procedures such as app upgrade and scale changes. I think a good enhancement could be to stop precisely at checkpoint time to prevent emission of spurious records. Today the behavior of 'cancel w/ savepoint' is at-least-once because the two operations aren't atomic. Earlier I had assumed that 'stop' would evolve in this direction but I suppose we could improve the atomicity of 'cancel /w savepoint' rather than implicating 'stop'. A different direction for 'stop' might be to improve the determinism of bounding a streaming job such that the stop point is well-understood in terms of the source. For example, stopping at a offset provided as a stop parameter. Today I suppose one would rely on external state to remember the stop point, e.g. FlinkKafkaConsumer010::setStartFromGroupOffsets. On Thu, Sep 14, 2017 at 1:03 AM, Ufuk Celebi wrote: > Hey Elias, > > sorry for the delay here. No, stop is not deprecated but not fully > implemented yet. One missing part is migration of the existing source > functions as you say. > > Let me pull in Till for more details on this. @Till: Is there more > missing than migrating the sources? > > Here is the PR and discussion for reference: > https://github.com/apache/flink/pull/750 > > I would also really love to see this fully implemented in Flink. I > don't expect this to happen for the upcoming 1.4 release though. > > =E2=80=93 Ufuk > > > On Wed, Sep 13, 2017 at 7:07 PM, Elias Levy > wrote: > > Anyone? > > > > On Mon, Sep 11, 2017 at 6:17 PM, Elias Levy > > > wrote: > >> > >> I was wondering about the status of the flink stop command. At first > >> blush it would seem as the preferable way to shutdown a Flink job, but > it > >> depends on StoppableFunction being implemented by sources and I notice > that > >> the Kafka source does not seem to implement it. In addition, the > command > >> does not -s --withSavepoint like cancel does. > >> > >> Is stop deprecated? > > > > > --001a113d840006df3d05592a1750 Content-Type: text/html; charset="UTF-8" Content-Transfer-Encoding: quoted-printable
I too am curious about stop vs cancel.=C2=A0 I'm = trying to understand the motivations a bit more.

The cur= rent behavior of stop is basically that the sources become bounded, leading= to the job winding down.

The interesting question= is how best to support 'planned' maintenance procedures such as ap= p upgrade and scale changes. =C2=A0 I think a good enhancement could be to = stop precisely at checkpoint time to prevent emission of spurious records.= =C2=A0 Today the behavior of 'cancel w/ savepoint' is at-least-once= because the two operations aren't atomic.=C2=A0 Earlier I had assumed = that 'stop' would evolve in this direction but I suppose we could i= mprove the atomicity of 'cancel /w savepoint' rather than implicati= ng 'stop'.

A different direction for '= stop' might be to improve the determinism of bounding a streaming job s= uch that the stop point is well-understood in terms of the source.=C2=A0 Fo= r example, stopping at a offset provided as a stop parameter. =C2=A0 Today = I suppose one would rely on external state to remember the stop point, e.g.= FlinkKafkaConsumer010::setStartFromGroupOffsets.

On Thu, Sep 14, 2017 at 1= :03 AM, Ufuk Celebi <uce@apache.org> wrote:
Hey Elias,

sorry for the delay here. No, stop is not deprecated but not fully
implemented yet. One missing part is migration of the existing source
functions as you say.

Let me pull in Till for more details on this. @Till: Is there more
missing than migrating the sources?

Here is the PR and discussion for reference:
https://github.com/apache/flink/pull/750

I would also really love to see this fully implemented in Flink. I
don't expect this to happen for the upcoming 1.4 release though.

=E2=80=93 Ufuk


On Wed, Sep 13, 2017 at 7:07 PM, Elias Levy <fearsome.lucidity@gmail.com> wrote:
> Anyone?
>
> On Mon, Sep 11, 2017 at 6:17 PM, Elias Levy <fearsome.lucidity@gmail.com>
> wrote:
>>
>> I was wondering about the status of the flink stop command.=C2=A0 = At first
>> blush it would seem as the preferable way to shutdown a Flink job,= but it
>> depends on StoppableFunction being implemented by sources and I no= tice that
>> the Kafka source does not seem to implement it.=C2=A0 In addition,= the command
>> does not -s=C2=A0 --withSavepoint like cancel does.
>>
>> Is stop deprecated?
>
>

--001a113d840006df3d05592a1750--