You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/03/28 11:58:50 UTC

[GitHub] [airflow] tirkarthi commented on issue #22141: Dump configurations in airflow scheduler logs based on the config it reads

tirkarthi commented on issue #22141:
URL: https://github.com/apache/airflow/issues/22141#issuecomment-1080560527


   Hi, I would like to work on this feature and this is my first patch. I looked into `airflow conf list` that has `show_config`  to print config. It doesn't seem to mask sensitive values. I wrote another function that initializes `AirflowConfigParser` and uses `as_dict` that masks sensitive values to read the masked config again using `read_dict`. I am not sure if the approach is correct. Patch with test as below
   
   https://github.com/apache/airflow/blob/ae6570ffefc124fd4ba7cf16648ce7dd21dc6aec/airflow/cli/commands/config_command.py#L23-L35
   
   Patch : 
   
   ```diff
   diff --git a/airflow/cli/commands/scheduler_command.py b/airflow/cli/commands/scheduler_command.py
   index bc6e983ee..4fae84f43 100644
   --- a/airflow/cli/commands/scheduler_command.py
   +++ b/airflow/cli/commands/scheduler_command.py
   @@ -16,6 +16,7 @@
    # under the License.
    
    """Scheduler command"""
   +import io
    import signal
    from multiprocessing import Process
    from typing import Optional
   @@ -24,6 +25,7 @@ import daemon
    from daemon.pidfile import TimeoutPIDLockFile
    
    from airflow import settings
   +from airflow.configuration import AirflowConfigParser, conf
    from airflow.jobs.scheduler_job import SchedulerJob
    from airflow.utils import cli as cli_utils
    from airflow.utils.cli import process_subdir, setup_locations, setup_logging, sigint_handler, sigquit_handler
   @@ -38,6 +40,16 @@ def _create_scheduler_job(args):
        return job
    
    
   +def _get_masked_config():
   +    parser = AirflowConfigParser(strict=False, interpolation=None)
   +    config_dict = conf.as_dict()
   +    parser.read_dict(config_dict)
   +
   +    with io.StringIO() as output:
   +        parser.write(output)
   +        return output.getvalue()
   +
   +
    def _run_scheduler_job(args):
        skip_serve_logs = args.skip_serve_logs
        job = _create_scheduler_job(args)
   @@ -53,6 +65,7 @@ def _run_scheduler_job(args):
    def scheduler(args):
        """Starts Airflow Scheduler"""
        print(settings.HEADER)
   +    print(_get_masked_config())
    
        if args.daemon:
            pid, stdout, stderr, log_file = setup_locations(
   diff --git a/tests/cli/commands/test_scheduler_command.py b/tests/cli/commands/test_scheduler_command.py
   index 00b851f55..8946e6c52 100644
   --- a/tests/cli/commands/test_scheduler_command.py
   +++ b/tests/cli/commands/test_scheduler_command.py
   @@ -15,6 +15,9 @@
    # KIND, either express or implied.  See the License for the
    # specific language governing permissions and limitations
    # under the License.
   +import contextlib
   +import io
   +import os
    import unittest
    from unittest import mock
    
   @@ -87,3 +90,14 @@ class TestSchedulerCommand(unittest.TestCase):
                    scheduler_command.scheduler(args)
                finally:
                    mock_process().terminate.assert_called()
   +
   +    @mock.patch.dict(os.environ, {}, clear=True)
   +    @mock.patch("airflow.cli.commands.scheduler_command.SchedulerJob")
   +    @mock.patch("airflow.cli.commands.scheduler_command.Process")
   +    def test_scheduler_print_config(self, mock_process, mock_scheduler_job):
   +        args = self.parser.parse_args(['scheduler'])
   +        with conf_vars({("core", "sql_alchemy_conn_cmd"): 'echo hello'}):
   +            with contextlib.redirect_stdout(io.StringIO()) as temp_stdout:
   +                scheduler_command.scheduler(args)
   +                output = temp_stdout.getvalue()
   +                assert "sql_alchemy_conn = < hidden >" in output
   
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org