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 <[email protected]> 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 <[email protected]> 
>> 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 
>>> <[email protected]> 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 
>>>> <[email protected]> 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 <[email protected]> wrote:
>>>>> 
>>>>> Anyone else? Last day before I close the vote off.
>>>>> 
>>>>> On Mon, Nov 27, 2017 at 12:32 PM, Driesprong, Fokko <[email protected]>
>>>>> wrote:
>>>>> 
>>>>>> +1
>>>>>> 
>>>>>> Op ma 27 nov. 2017 om 21:24 schreef Bolke de Bruin <[email protected]>
>>>>>> 
>>>>>>> +1, binding
>>>>>>> 
>>>>>>> Let’s get it out!
>>>>>>> 
>>>>>>> Sent from my iPhone
>>>>>>> 
>>>>>>>> On 27 Nov 2017, at 20:32, Chris Riccomini <[email protected]>
>>>>>> 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