K, I'm going to let this vote hang here until we get confirmation as to
what's going on.

@bolke/@ash, thanks for looking into this!

On Thu, Nov 30, 2017 at 10:23 AM, Bolke de Bruin <bdbr...@gmail.com> wrote:

> Ok, I think I can confirm the issue. I don’t see any output either with a
> clean install. I’ll investigate a bit further.
>
> *sigh*
>
> Bolke
>
>
> > On 30 Nov 2017, at 19:13, Ash Berlin-Taylor <ash_airflowlist@firemirror.
> com> wrote:
> >
> > Yes, that is in my logging config.
> >
> > I think I agree about airflow.task.raw
> >
> > -ash
> >
> >
> >> On 30 Nov 2017, at 18:06, Joy Gao <j...@wepay.com> wrote:
> >>
> >> Hi Ash,
> >>
> >> Can you verify that your logging config has root logger set to
> 'console'?
> >> Something like:
> >>
> >>       '': {
> >>           'handlers': ['console'],
> >>           'level': LOG_LEVEL
> >>       },
> >>
> >>
> >> That was the issue we had because we attempted to set *all* handlers to
> >> airflow.task at first.
> >>
> >> I don't think we gain anything by calling airflow.task.raw logger, so I
> >> have a pending PR that removes airflow.task.raw and just uses
> airflow.task.
> >>
> >>
> >>
> >> On Thu, Nov 30, 2017 at 9:24 AM, Bolke de Bruin <bdbr...@gmail.com>
> wrote:
> >>
> >>> We saw this issue around RC2, so I’m surprised that it turns up again.
> We
> >>> discussed it at lengths with Fokko and Joy. It might be that we somehow
> >>> missed a commit, but I need to double check a from stock install. We
> are
> >>> running 1.9 in production and we don’t have any logging issues.
> >>>
> >>> B.
> >>>
> >>> Verstuurd vanaf mijn iPad
> >>>
> >>>> Op 30 nov. 2017 om 18:17 heeft Chris Riccomini <criccom...@apache.org
> >
> >>> het volgende geschreven:
> >>>>
> >>>> @bolke/@fokko, thoughts?
> >>>>
> >>>> 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.
> >>>>
> >>>> On Thu, Nov 30, 2017 at 9:08 AM, Ash Berlin-Taylor <
> >>>> ash_airflowl...@firemirror.com> wrote:
> >>>>
> >>>>> 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.
> >>>>>
> >>>>> There's another problem after applying a local fix for
> >>>>> `handler.set_context()`, to do with try_numbers:
> >>>>>
> >>>>> 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
> >>>>>
> >>>>> 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
> >>>>>
> >>>>> 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.
> >>>>>
> >>>>> At this point I might be a bit deep and might need to look with fresh
> >>> eyes
> >>>>> and a clean slate tomorrow. Someone confirming if they do or don't
> see
> >>> logs
> >>>>> from inside an operator would help me.
> >>>>>
> >>>>> -ash
> >>>>>
> >>>>>
> >>>>>> On 30 Nov 2017, at 17:03, Chris Riccomini <criccom...@apache.org>
> >>> wrote:
> >>>>>>
> >>>>>> @Ash, do you think this issue is blocking for 1.9.0?
> >>>>>>
> >>>>>> On Thu, Nov 30, 2017 at 8:22 AM, Ash Berlin-Taylor <
> >>>>>> ash_airflowl...@firemirror.com> wrote:
> >>>>>>
> >>>>>>> Ah, I think I've got it.
> >>>>>>>
> >>>>>>> The problem is that airflow.task.raw doesn't have an explicit
> config
> >>>>>>> anymore(?), so when we do:
> >>>>>>>
> >>>>>>> log = logging.getLogger('airflow.task.raw')
> >>>>>>>
> >>>>>>> and look at its handlers, it doesn't have any. But
> log.parent.handlers
> >>>>>>> _does_ have them.
> >>>>>>>
> >>>>>>> A few options to fix this:
> >>>>>>>
> >>>>>>> - 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)
> >>>>>>>
> >>>>>>> -ash
> >>>>>>>
> >>>>>>>
> >>>>>>>> On 30 Nov 2017, at 16:02, Ash Berlin-Taylor
> >>>>> <ash_airflowlist@firemirror.
> >>>>>>> com> wrote:
> >>>>>>>>
> >>>>>>>> I'm fairly sure I'm back to stock config now. Here is the DAG I am
> >>>>>>> testing with:
> >>>>>>>>
> >>>>>>>> from datetime import datetime
> >>>>>>>> from airflow import DAG
> >>>>>>>> from airflow.operators.python_operator import PythonOperator
> >>>>>>>>
> >>>>>>>> dag_args = {
> >>>>>>>> "start_date": datetime(2017, 11, 20),
> >>>>>>>> }
> >>>>>>>> dag = DAG("tests", default_args=dag_args)
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> def print_stuff(ti, **kwargs):
> >>>>>>>> print("Hi from", __file__)
> >>>>>>>> ti.log.error("Hello")
> >>>>>>>>
> >>>>>>>> with dag:
> >>>>>>>> PythonOperator(
> >>>>>>>>    task_id="test-logging",
> >>>>>>>>    dag=dag,
> >>>>>>>>    python_callable=print_stuff,
> >>>>>>>>    provide_context=True,
> >>>>>>>> )
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 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
> >>> the
> >>>>>>> config before it uses it, and this is the config I'm getting:
> >>>>>>>>
> >>>>>>>> {'disable_existing_loggers': False,
> >>>>>>>> 'formatters': {'airflow.processor': {'format': '[%(asctime)s] '
> >>>>>>>>
> >>>>>>> '{%(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}},
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> If I edit run() in in bin/cli.py to print the log.handlers under
> this
> >>>>>>> circumstance I get an empty list [].
> >>>>>>>>
> >>>>>>>> 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.
> >>>>>>>>
> >>>>>>>> -ash
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>> On 30 Nov 2017, at 15:38, Bolke de Bruin <bdbr...@gmail.com>
> wrote:
> >>>>>>>>>
> >>>>>>>>> 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.
> >>>>>>>>>
> >>>>>>>>> Bolke.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>> On 30 Nov 2017, at 16:27, Ash Berlin-Taylor <
> >>>>>>> ash_airflowl...@firemirror.com> wrote:
> >>>>>>>>>>
> >>>>>>>>>> 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
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> log = logging.getLogger('airflow.task')
> >>>>>>>>>> if args.raw:
> >>>>>>>>>> log = logging.getLogger('airflow.task.raw')
> >>>>>>>>>>
> >>>>>>>>>> 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
> >>>>>>>>>>
> >>>>>>>>>> The problem seems to be that airflow.task.raw is no longer
> >>> configured
> >>>>>>> explicitly, so it's log.handlers is an empty list.
> >>>>>>>>>>
> >>>>>>>>>> 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.
> >>>>>>>>>>
> >>>>>>>>>> Does this ring any bells to anyone?
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>> On 30 Nov 2017, at 15:18, Ash Berlin-Taylor <
> >>>>>>> ash_airflowl...@firemirror.com> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> 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()`
> >>>>>>>>>>>
> >>>>>>>>>>> Anyone else noticed similar, or is it something specific to our
> >>>>>>> environment?
> >>>>>>>>>>>
> >>>>>>>>>>> -ash
> >>>>>>>>>>>
> >>>>>>>>>>>> On 30 Nov 2017, at 09:52, Ash Berlin-Taylor <
> >>>>>>> ash_airflowl...@firemirror.com> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>> 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.
> >>>>>>>>>>>>
> >>>>>>>>>>>> +0.5 (non binding)
> >>>>>>>>>>>>
> >>>>>>>>>>>> -ash
> >>>>>>>>>>>>
> >>>>>>>>>>>>> On 29 Nov 2017, at 18:31, Chris Riccomini <
> >>> criccom...@apache.org>
> >>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Anyone else? Last day before I close the vote off.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, Nov 27, 2017 at 12:32 PM, Driesprong, Fokko
> >>>>>>> <fo...@driesprong.frl>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> +1
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Op ma 27 nov. 2017 om 21:24 schreef Bolke de Bruin <
> >>>>>>> bdbr...@gmail.com>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> +1, binding
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Let’s get it out!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Sent from my iPhone
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 27 Nov 2017, at 20:32, Chris Riccomini <
> >>>>> criccom...@apache.org
> >>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hey all,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Airflow 1.9.0 RC4 is available at:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> https://dist.apache.org/repos/
> dist/dev/incubator/airflow/1.
> >>>>>>> 9.0rc4/
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Public keys are available at:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> https://dist.apache.org/repos/dist/release/incubator/
> >>> airflow/
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> The release contains the following JIRAs:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> *https://issues.apache.org/jira/browse/AIRFLOW-1839
> >>>>>>>>>>>>>>>> <https://issues.apache.org/jira/browse/AIRFLOW-1839>*
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Along with all JIRAs that were in 1.9.0RC3 (see previous
> VOTE
> >>>>>>> email for
> >>>>>>>>>>>>>>>> full list).
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>> Chris
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>
> >>>>>
> >>>
> >>>
> >
>
>

Reply via email to