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 CAB22200D4C for ; Thu, 30 Nov 2017 18:24:50 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id C9238160C01; Thu, 30 Nov 2017 17:24:50 +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 C0B3E160BF6 for ; Thu, 30 Nov 2017 18:24:49 +0100 (CET) Received: (qmail 2094 invoked by uid 500); 30 Nov 2017 17:24:48 -0000 Mailing-List: contact dev-help@airflow.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@airflow.incubator.apache.org Delivered-To: mailing list dev@airflow.incubator.apache.org Received: (qmail 2082 invoked by uid 99); 30 Nov 2017 17:24:48 -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, 30 Nov 2017 17:24:48 +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 EB60E1A0AC8 for ; Thu, 30 Nov 2017 17:24:47 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.879 X-Spam-Level: X-Spam-Status: No, score=0.879 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, KAM_NUMSUBJECT=0.5, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, 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 P-J7sKf28Yml for ; Thu, 30 Nov 2017 17:24:45 +0000 (UTC) Received: from mail-wm0-f50.google.com (mail-wm0-f50.google.com [74.125.82.50]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id 0983C5F6CD for ; Thu, 30 Nov 2017 17:24:45 +0000 (UTC) Received: by mail-wm0-f50.google.com with SMTP id g75so14615468wme.0 for ; Thu, 30 Nov 2017 09:24:45 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20161025; h=from:content-transfer-encoding:mime-version:date:subject:message-id :references:in-reply-to:to; bh=mH7Y9SoucOWy4Yn/hfwS5CgscBGM5zDG4Hqe0Pxi4EE=; b=L9qcV4Q7Pst8cmtD2nlhImuv0Q2anRxLrvq6O/HIvi95YajAQAvQ8EGgeQBcGWMVRE JcufYWR3TrLspHmZD3gpOZ5G7vRexdYSt877BVzeWZvp3rMbIzCgWUaUWlMd23pJEtp3 ujR3hMQI3LPWBSRuxaQdeFdsfCmWHyFZjQRqP0CAxV97tIupNXnxwasV1ZyN1K7vENXX iW8yze5W1cA3uyUBv4KF33IUD7GpPglRKndySw9iGua/0sBI7r88rz7QanwFlgL8BSwc xUTeL4PlAZiGMZFa0tDO5UKxj8zLbIp8VkKSCciu3oGjzYSoVuGmoydrNstm5x68pAzl 0RAQ== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:from:content-transfer-encoding:mime-version:date :subject:message-id:references:in-reply-to:to; bh=mH7Y9SoucOWy4Yn/hfwS5CgscBGM5zDG4Hqe0Pxi4EE=; b=hv8C7pSYveIj5GXAEcN4nvcgLOYYXG1H9LEdJL4b38KZifUdNINbmjZbrEpg1tnfkn 6Obt+1mOBG5UmQvT6eH3qt2shr8wBpHONWX38bMmUFZxaMB0wevyFEJoX/6EHDyaDtiv oN058WBE34drM+01/6KPVdhje1dpJVbvVL2yyI/P/slqQMpnxg0YK9jg+eLw4Z6pFrMb SlZ7x8jUPImUgCBxAZ0h2iGuH0CUWACRaYXpJuhtLJQCG4UTZ6PA8RtgCPfQW/VUZNdy jG/2awWVIr4dz5lIzY7QXobk97r+t7CLoxHVz/SWNdoaZQrGk45SXZeTfyeoqNJPsWVb xpFA== X-Gm-Message-State: AJaThX6zezygvD4yBdi5cIxwsq3KdX74g2GdZXmyieG9CVnGvjiL87K2 YWFX05Xhj+vFQyL8LbHJDVRTU+Q2JA1r X-Google-Smtp-Source: AGs4zMZHf09SL8uMaadISDCd6B4NW/D2WlLL6jrnnj8VukE67VaN4U7vZ7VbAn/GmUr29aN3jcKKcw== X-Received: by 10.80.205.25 with SMTP id z25mr13827328edi.77.1512062683978; Thu, 30 Nov 2017 09:24:43 -0800 (PST) Received: from [10.0.1.99] (f248061.upc-f.chello.nl. [80.56.248.61]) by smtp.gmail.com with ESMTPSA id x49sm4182838edb.76.2017.11.30.09.24.42 for (version=TLS1_2 cipher=ECDHE-RSA-AES128-GCM-SHA256 bits=128/128); Thu, 30 Nov 2017 09:24:43 -0800 (PST) From: Bolke de Bruin Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable Mime-Version: 1.0 (1.0) Date: Thu, 30 Nov 2017 18:24:42 +0100 Subject: Re: [VOTE] Airflow 1.9.0rc4 Message-Id: <38E294B1-7641-459E-B642-A3E19B6341C6@gmail.com> References: <70326A1F-46C0-450A-AC34-194722EA2C56@gmail.com> <754FC338-773C-46FC-BC33-B35630FC39EC@firemirror.com> <86ACB8DC-F946-4572-999A-30179567F5A7@firemirror.com> <2787914E-9ADF-49C7-A8C3-4D23EAB83E47@firemirror.com> <51B99D9F-3DBE-452B-A04A-1852A90DD8D4@gmail.com> <69ED3101-0DEC-41C3-B6CA-DD5E9BE2715B@firemirror.com> <70828941-8CEA-4559-B415-743368A06448@firemirror.com> In-Reply-To: To: dev@airflow.incubator.apache.org X-Mailer: iPad Mail (15B202) archived-at: Thu, 30 Nov 2017 17:24:51 -0000 We saw this issue around RC2, so I=E2=80=99m surprised that it turns up agai= n. We discussed it at lengths with Fokko and Joy. It might be that we someho= w missed a commit, but I need to double check a from stock install. We are r= unning 1.9 in production and we don=E2=80=99t have any logging issues. B. Verstuurd vanaf mijn iPad > Op 30 nov. 2017 om 18:17 heeft Chris Riccomini het= volgende geschreven: >=20 > @bolke/@fokko, thoughts? >=20 > As an aside, we had an issue seeing task logs, but were able to fix it by > modifying our logging config (and perhaps patching some stuff that we > merged into 1.9.0, don't recall). William Pursell/Joy Gao might be able to= > comment more. >=20 > On Thu, Nov 30, 2017 at 9:08 AM, Ash Berlin-Taylor < > ash_airflowlist@firemirror.com> wrote: >=20 >> If anyone else is seeing this (and I think I'm on stock configuration now= ) >> yes I do, as I'm not seeing _any_ logs form task operators appearing. >>=20 >> There's another problem after applying a local fix for >> `handler.set_context()`, to do with try_numbers: >>=20 >> The `airflow run tests test-logging 2017-11-26T00:00:00 --local -sd >> /usr/local/airflow/dags/example/.py` command sends logs to .../14.log >>=20 >> The `airflow run tests test-logging 2017-11-26T00:00:00 --job_id 33 --raw= >> -sd /usr/local/airflow/dags/example/.py` command (that the previous >> command invokes) sends logs to .../15.log >>=20 >> However the UI doesn't know about try 15 (yes try 15. I've been running >> this a bit) so any future runs will write to this file. >>=20 >> At this point I might be a bit deep and might need to look with fresh eye= s >> and a clean slate tomorrow. Someone confirming if they do or don't see lo= gs >> from inside an operator would help me. >>=20 >> -ash >>=20 >>=20 >>> On 30 Nov 2017, at 17:03, Chris Riccomini wrote:= >>>=20 >>> @Ash, do you think this issue is blocking for 1.9.0? >>>=20 >>> On Thu, Nov 30, 2017 at 8:22 AM, Ash Berlin-Taylor < >>> ash_airflowlist@firemirror.com> wrote: >>>=20 >>>> Ah, I think I've got it. >>>>=20 >>>> The problem is that airflow.task.raw doesn't have an explicit config >>>> anymore(?), so when we do: >>>>=20 >>>> log =3D logging.getLogger('airflow.task.raw') >>>>=20 >>>> and look at its handlers, it doesn't have any. But log.parent.handlers >>>> _does_ have them. >>>>=20 >>>> A few options to fix this: >>>>=20 >>>> - I can add an explicit handler for .task.raw in my logging config. >>>> - We can always call set_context on airflow.task instead of >>>> airflow.task.raw >>>> - We walk up the .parent links if .propagate is True (possibly stopping= >>>> once we find any handlers) >>>>=20 >>>> -ash >>>>=20 >>>>=20 >>>>> On 30 Nov 2017, at 16:02, Ash Berlin-Taylor >> >>> com> wrote: >>>>>=20 >>>>> I'm fairly sure I'm back to stock config now. Here is the DAG I am >>>> testing with: >>>>>=20 >>>>> from datetime import datetime >>>>> from airflow import DAG >>>>> from airflow.operators.python_operator import PythonOperator >>>>>=20 >>>>> dag_args =3D { >>>>> "start_date": datetime(2017, 11, 20), >>>>> } >>>>> dag =3D DAG("tests", default_args=3Ddag_args) >>>>>=20 >>>>>=20 >>>>> def print_stuff(ti, **kwargs): >>>>> print("Hi from", __file__) >>>>> ti.log.error("Hello") >>>>>=20 >>>>> with dag: >>>>> PythonOperator( >>>>> task_id=3D"test-logging", >>>>> dag=3Ddag, >>>>> python_callable=3Dprint_stuff, >>>>> provide_context=3DTrue, >>>>> ) >>>>>=20 >>>>>=20 >>>>> And I don't see either of the "Hi from" or "Hello" lines in the >>>> resulting log file. I have edited airflow/logging_config.py to print th= e >>>> config before it uses it, and this is the config I'm getting: >>>>>=20 >>>>> {'disable_existing_loggers': False, >>>>> 'formatters': {'airflow.processor': {'format': '[%(asctime)s] ' >>>>>=20 >>>> '{%(filename)s:%(lineno)d} ' >>>>> '%(levelname)s - >>>> %(message)s'}, >>>>> 'airflow.task': {'format': '[%(asctime)s] ' >>>>> '{%(filename)s:%(lineno)d} ' >>>>> '%(levelname)s - >>>> %(message)s'}}, >>>>> 'handlers': {'console': {'class': 'logging.StreamHandler', >>>>> 'formatter': 'airflow.task', >>>>> 'stream': 'ext://sys.stdout'}, >>>>> 'file.processor': {'base_log_folder': >>>> '/usr/local/airflow/logs/scheduler', >>>>> 'class': 'airflow.utils.log.file_ >>>> processor_handler.FileProcessorHandler', >>>>> 'filename_template': '{{ filename >>>> }}.log', >>>>> 'formatter': 'airflow.processor'}, >>>>> 'file.task': {'base_log_folder': >> '/usr/local/airflow/logs/', >>>>> 'class': 'airflow.utils.log.file_task_ >>>> handler.FileTaskHandler', >>>>> 'filename_template': '{{ ti.dag_id }}/{{ ' >>>>> 'ti.task_id }}/{{ ts >>>> }}/{{ ' >>>>> 'try_number }}.log', >>>>> 'formatter': 'airflow.task'}}, >>>>> 'loggers': {'': {'handlers': ['console'], 'level': 'INFO'}, >>>>> 'airflow': {'handlers': ['console'], >>>>> 'level': 'INFO', >>>>> 'propagate': False}, >>>>> 'airflow.processor': {'handlers': ['file.processor'], >>>>> 'level': 'INFO', >>>>> 'propagate': True}, >>>>> 'airflow.task': {'handlers': ['file.task'], >>>>> 'level': 'INFO', >>>>> 'propagate': False}, >>>>> 'airflow.task_runner': {'handlers': ['file.task'], >>>>> 'level': 'INFO', >>>>> 'propagate': True}}, >>>>>=20 >>>>>=20 >>>>> If I edit run() in in bin/cli.py to print the log.handlers under this >>>> circumstance I get an empty list []. >>>>>=20 >>>>> I guess since we don't specify 'airflow.task.raw' in the loggers that >> it >>>> should go up to the next level in 'airflow.task'? Except it doesn't >> seem to >>>> be for me. I'm digging further. >>>>>=20 >>>>> -ash >>>>>=20 >>>>>=20 >>>>>=20 >>>>>> On 30 Nov 2017, at 15:38, Bolke de Bruin wrote: >>>>>>=20 >>>>>> airflow.task.raw points to airflow.task by default, which does have a= >>>> handler and that is the one that is travelled. And yes we know the code= >> is >>>> incorrect in this area, but the default does work correctly. >>>>>>=20 >>>>>> Bolke. >>>>>>=20 >>>>>>=20 >>>>>>> On 30 Nov 2017, at 16:27, Ash Berlin-Taylor < >>>> ash_airflowlist@firemirror.com> wrote: >>>>>>>=20 >>>>>>> Okay I think the problem (if not the cause) is around here: >>>> https://github.com/apache/incubator-airflow/blob/master/ >>>> airflow/bin/cli.py#L366-L376 >>>>>>>=20 >>>>>>>=20 >>>>>>>=20 >>>>>>> log =3D logging.getLogger('airflow.task') >>>>>>> if args.raw: >>>>>>> log =3D logging.getLogger('airflow.task.raw') >>>>>>>=20 >>>>>>> for handler in log.handlers: >>>>>>> try: >>>>>>> handler.set_context(ti) >>>>>>> except AttributeError: >>>>>>> # Not all handlers need to have context passed in so we ignore >>>>>>> # the error when handlers do not have set_context defined. >>>>>>> pass >>>>>>>=20 >>>>>>> The problem seems to be that airflow.task.raw is no longer configure= d >>>> explicitly, so it's log.handlers is an empty list. >>>>>>>=20 >>>>>>> This in turn means that any log calls end up not writing anything as= >>>> our FileTaskHandler's self.handler property is None, meaning it doesn't= >>>> emit anything. >>>>>>>=20 >>>>>>> Does this ring any bells to anyone? >>>>>>>=20 >>>>>>>=20 >>>>>>>> On 30 Nov 2017, at 15:18, Ash Berlin-Taylor < >>>> ash_airflowlist@firemirror.com> wrote: >>>>>>>>=20 >>>>>>>> Hmmm is anyone else seeing logs from inside operators? Right now >>>> we're not getting any logs out of operators. I'm in the process of >>>> debugging to work out if it's our logging config or something in >> Airflow, >>>> but right now I don't seem to get any output from the operator itself, >>>> neither print(), or `ti.log.info()` >>>>>>>>=20 >>>>>>>> Anyone else noticed similar, or is it something specific to our >>>> environment? >>>>>>>>=20 >>>>>>>> -ash >>>>>>>>=20 >>>>>>>>> On 30 Nov 2017, at 09:52, Ash Berlin-Taylor < >>>> ash_airflowlist@firemirror.com> wrote: >>>>>>>>>=20 >>>>>>>>> We're running more tests (day of meetings meant we couldn't do as >>>> many as I'd have liked yesterday) but looking good so far. >>>>>>>>>=20 >>>>>>>>> +0.5 (non binding) >>>>>>>>>=20 >>>>>>>>> -ash >>>>>>>>>=20 >>>>>>>>>> On 29 Nov 2017, at 18:31, Chris Riccomini = >>>> wrote: >>>>>>>>>>=20 >>>>>>>>>> Anyone else? Last day before I close the vote off. >>>>>>>>>>=20 >>>>>>>>>> On Mon, Nov 27, 2017 at 12:32 PM, Driesprong, Fokko >>>> >>>>>>>>>> wrote: >>>>>>>>>>=20 >>>>>>>>>>> +1 >>>>>>>>>>>=20 >>>>>>>>>>> Op ma 27 nov. 2017 om 21:24 schreef Bolke de Bruin < >>>> bdbruin@gmail.com> >>>>>>>>>>>=20 >>>>>>>>>>>> +1, binding >>>>>>>>>>>>=20 >>>>>>>>>>>> Let=E2=80=99s get it out! >>>>>>>>>>>>=20 >>>>>>>>>>>> Sent from my iPhone >>>>>>>>>>>>=20 >>>>>>>>>>>>> On 27 Nov 2017, at 20:32, Chris Riccomini < >> criccomini@apache.org >>>>>=20 >>>>>>>>>>> wrote: >>>>>>>>>>>>>=20 >>>>>>>>>>>>> Hey all, >>>>>>>>>>>>>=20 >>>>>>>>>>>>> I have cut Airflow 1.9.0 RC4. This email is calling a vote on >> the >>>>>>>>>>>> release, >>>>>>>>>>>>> which will last for 72 hours. Consider this my (binding) +1. >>>>>>>>>>>>>=20 >>>>>>>>>>>>> Airflow 1.9.0 RC4 is available at: >>>>>>>>>>>>>=20 >>>>>>>>>>>>> https://dist.apache.org/repos/dist/dev/incubator/airflow/1. >>>> 9.0rc4/ >>>>>>>>>>>>>=20 >>>>>>>>>>>>> apache-airflow-1.9.0rc4+incubating-source.tar.gz is a source >>>> release >>>>>>>>>>> that >>>>>>>>>>>>> comes with INSTALL instructions. >>>>>>>>>>>>> apache-airflow-1.9.0rc4+incubating-bin.tar.gz is the binary >>>> Python >>>>>>>>>>>> "sdist" >>>>>>>>>>>>> release. >>>>>>>>>>>>>=20 >>>>>>>>>>>>> Public keys are available at: >>>>>>>>>>>>>=20 >>>>>>>>>>>>> https://dist.apache.org/repos/dist/release/incubator/airflow/ >>>>>>>>>>>>>=20 >>>>>>>>>>>>> The release contains the following JIRAs: >>>>>>>>>>>>>=20 >>>>>>>>>>>>> *https://issues.apache.org/jira/browse/AIRFLOW-1839 >>>>>>>>>>>>> * >>>>>>>>>>>>>=20 >>>>>>>>>>>>> Along with all JIRAs that were in 1.9.0RC3 (see previous VOTE >>>> email for >>>>>>>>>>>>> full list). >>>>>>>>>>>>>=20 >>>>>>>>>>>>> Cheers, >>>>>>>>>>>>> Chris >>>>>>>>>>>>=20 >>>>>>>>>>>=20 >>>>>>>>>=20 >>>>>>>>=20 >>>>>>>=20 >>>>>>=20 >>>>>=20 >>>>=20 >>>>=20 >>=20 >>=20