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 2020/11/26 05:44:00 UTC

[GitHub] [airflow] dimberman opened a new pull request #12638: bash-operator-fix

dimberman opened a new pull request #12638:
URL: https://github.com/apache/airflow/pull/12638


   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r530847419



##########
File path: airflow/operators/bash.py
##########
@@ -157,14 +163,37 @@ def pre_exec():
 
             self.log.info('Running command: %s', self.bash_command)
 
-            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
-                ['bash', "-c", self.bash_command],
-                stdout=PIPE,
-                stderr=STDOUT,
-                cwd=tmp_dir,
-                env=env,
-                preexec_fn=pre_exec,
-            )
+            if isinstance(self.bash_command) == str and not _is_bash_script(self.bash_command):

Review comment:
       This PR needs some tests too @dimberman 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] closed pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #12638:
URL: https://github.com/apache/airflow/pull/12638


   


-- 
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r530845591



##########
File path: airflow/operators/bash.py
##########
@@ -95,10 +101,16 @@ class BashOperator(BaseOperator):
 
         bash_task = BashOperator(
             task_id="bash_task",
-            bash_command='echo "here is the message: \'$message\'"',
+            bash_command=['echo', "here is the message: \'$message\'"],
             env={'message': '{{ dag_run.conf["message"] if dag_run else "" }}'},
         )
 
+    We also highly recommend against using [\"bash\", \"-c\"... in your command as that command will negate

Review comment:
       Don't need `\` here




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb edited a comment on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb edited a comment on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778297656


   I wonder if we could have a "short" form version that fits in with the taskflow api:
   
   ```python
   @dag
   def my_workflow():
   
       @task
       def munge_output(line):
           return line.upper()
   
       # This is an operator/task that is run, not a static output.
       date = run_command(["date", "--iso=s"])
       munge_output(date)
   ```
   
   is the equivalent of
   
   ```python
   with DAG(dag_id="my_workflow"):
   
       def munge_output(line):
           return line.upper()
   
       # This is an operator/task that is run, not a static output.
       date_op = RunCommandOperator(
           command=["date", "--iso=s"],
           do_xcom_push=True,
       )
   
       munge_op = PythonOperator(
           task_id='munge_output',
           op_kwargs={'line': '{{ ti.xcom_pull('date') }}'}
       )
   ```


----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb edited a comment on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb edited a comment on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778297656


   I wonder if we could have a "short" form version that fits in with the taskflow api:
   
   ```python
   @dag
   def my_workflow():
   
       @task
       def munge_output(line):
           return line.upper()
   
       date = run_command(["date", "--iso=s"])
       munge_output(date)
   ```


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778296333


   Personally `RunCommandOperator` feels a bit better 


----------------------------------------------------------------
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.

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



[GitHub] [airflow] dstandish commented on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
dstandish commented on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-777877021


   @dimberman this one may be of interest: https://github.com/apache/airflow/pull/13423
   
   just stumbled on to the present pr... #13423 pulls the "run command" logic into a hook, and requires passing as list


----------------------------------------------------------------
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.

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



[GitHub] [airflow] TheRook commented on a change in pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
TheRook commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r531095496



##########
File path: airflow/operators/bash.py
##########
@@ -157,14 +163,37 @@ def pre_exec():
 
             self.log.info('Running command: %s', self.bash_command)
 
-            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
-                ['bash', "-c", self.bash_command],
-                stdout=PIPE,
-                stderr=STDOUT,
-                cwd=tmp_dir,
-                env=env,
-                preexec_fn=pre_exec,
-            )
+            if isinstance(self.bash_command) == str and not _is_bash_script(self.bash_command):
+                warnings.warn(
+                    "Warning: Using a string in the BashOperator leaves your system open to bash injection "
+                    "attacks via escape strings. Please use a list[str] instead."
+                )
+                self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                    ['bash', "-c", self.bash_command],
+                    stdout=PIPE,
+                    stderr=STDOUT,
+                    cwd=tmp_dir,
+                    env=env,
+                    preexec_fn=pre_exec,
+                )
+            elif isinstance(self.bash_command) == list:
+                if (
+                    len(self.bash_command) >= 2
+                    and self.bash_command[0] == "bash"
+                    and self.bash_command[1] == "-c"
+                ):

Review comment:
       This is good, they should get hit with a warning that is functionally equivalent and still leads to evaluation of bash entities. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] michalslowikowski00 commented on a change in pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
michalslowikowski00 commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r530845944



##########
File path: airflow/operators/bash.py
##########
@@ -157,14 +163,31 @@ def pre_exec():
 
             self.log.info('Running command: %s', self.bash_command)
 
-            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
-                ['bash', "-c", self.bash_command],
-                stdout=PIPE,
-                stderr=STDOUT,
-                cwd=tmp_dir,
-                env=env,
-                preexec_fn=pre_exec,
-            )
+            if isinstance(self.bash_command) == str and not _is_bash_script(self.bash_command):
+                warnings.warn(
+                    "Warning: Using a string in the BashOperator leaves your system open to bash injection "
+                    "attacks via escape strings. Please use a list[str] instead.\n"
+                    "\n"
+                    "We also highly recommend against using [\"bash\", \"-c\"... in your command as"
+                    "that command will essentially negate any security that Popen can offer you."
+                )
+                self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                    ['bash', "-c", self.bash_command],
+                    stdout=PIPE,
+                    stderr=STDOUT,
+                    cwd=tmp_dir,
+                    env=env,
+                    preexec_fn=pre_exec,
+                )
+            elif isinstance(self.bash_command) == list:

Review comment:
       ```suggestion
               elif isinstance(self.bash_command, list):
   ```
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778297656


   I wonder if we could have a "short" form version that fits in with the taskflow api:
   
   ```python
   @dag
   def my_workflow():
   
       @task
       def munge_output(line):
           return line.upper()
   
       date = run_command("date --iso=s")
       munge_output(date)
   ```


----------------------------------------------------------------
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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-809824799


   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


-- 
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.

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



[GitHub] [airflow] TheRook commented on a change in pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
TheRook commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r531096151



##########
File path: airflow/operators/bash.py
##########
@@ -95,10 +101,16 @@ class BashOperator(BaseOperator):
 
         bash_task = BashOperator(
             task_id="bash_task",
-            bash_command='echo "here is the message: \'$message\'"',
+            bash_command=['echo', "here is the message: \'$message\'"],
             env={'message': '{{ dag_run.conf["message"] if dag_run else "" }}'},
         )
 
+    We also highly recommend against using [\"bash\", \"-c\"... in your command as that command will negate

Review comment:
       Perhaps we can link to some docs here?  I am willing to write up a page on injection in general, and why parallelization is a good tool for the job. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778253195


   @dstandish Yeah, we're going to close this, and create a separate operator -- leaving the Bash as _actually running bash_, and create a new one to called something like "RunCommandOperator" and make that the "default" in all the docs over BashOperator.
   
   Ideas for a better/shorted name appreciated.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r531467290



##########
File path: airflow/operators/bash.py
##########
@@ -157,14 +163,37 @@ def pre_exec():
 
             self.log.info('Running command: %s', self.bash_command)
 
-            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
-                ['bash', "-c", self.bash_command],
-                stdout=PIPE,
-                stderr=STDOUT,
-                cwd=tmp_dir,
-                env=env,
-                preexec_fn=pre_exec,
-            )
+            if isinstance(self.bash_command) == str and not _is_bash_script(self.bash_command):
+                warnings.warn(
+                    "Warning: Using a string in the BashOperator leaves your system open to bash injection "
+                    "attacks via escape strings. Please use a list[str] instead."
+                )
+                self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                    ['bash', "-c", self.bash_command],
+                    stdout=PIPE,
+                    stderr=STDOUT,
+                    cwd=tmp_dir,
+                    env=env,
+                    preexec_fn=pre_exec,
+                )
+            elif isinstance(self.bash_command) == list:
+                if (
+                    len(self.bash_command) >= 2
+                    and self.bash_command[0] == "bash"
+                    and self.bash_command[1] == "-c"
+                ):

Review comment:
       Yes, running bash is the point of the operator - it's called a BashOperetor, not ExecCommandOperator ;) 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r532120979



##########
File path: airflow/operators/bash.py
##########
@@ -95,10 +101,16 @@ class BashOperator(BaseOperator):
 
         bash_task = BashOperator(
             task_id="bash_task",
-            bash_command='echo "here is the message: \'$message\'"',
+            bash_command=['echo', "here is the message: \'$message\'"],
             env={'message': '{{ dag_run.conf["message"] if dag_run else "" }}'},
         )
 
+    We also highly recommend against using [\"bash\", \"-c\"... in your command as that command will negate

Review comment:
       Doc PRs are always appreciated!




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r530844550



##########
File path: airflow/operators/bash.py
##########
@@ -19,16 +19,22 @@
 
 import os
 import signal
+import warnings
 from subprocess import PIPE, STDOUT, Popen
 from tempfile import TemporaryDirectory, gettempdir
-from typing import Dict, Optional
+from typing import Dict, List, Optional, Union
 
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
 from airflow.utils.operator_helpers import context_to_airflow_vars
 
 
+def _is_bash_script(input_string: str):
+    input_list = input_string.split(" ")
+    return len(input_list) == 1 and len(input_string) > 2 and input_string[3:] == ".sh"

Review comment:
       This check doesn't seem right it only also for `a.sh` not `longer.sh`




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778294938


   > > Ideas for a better/shorted name appreciated
   > 
   > well `CommandOperator` is shorter if not better :)
   > 
   > there's also `SubprocessOperator`
   
   > 
   > or `PopenOperator`
   
   Popen probably only makes sense if you are a python developer and know what Popen is.
   
   Of those I think CommandOperator (or ExecOperator as you had before you edited it?).


----------------------------------------------------------------
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.

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



[GitHub] [airflow] dstandish removed a comment on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
dstandish removed a comment on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778264183


   Lastly `ExecOperator`


----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb commented on a change in pull request #12638: bash-operator-fix

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#discussion_r530821843



##########
File path: airflow/operators/bash.py
##########
@@ -19,16 +19,22 @@
 
 import os
 import signal
+import warnings
 from subprocess import PIPE, STDOUT, Popen
 from tempfile import TemporaryDirectory, gettempdir
-from typing import Dict, Optional
+from typing import Dict, List, Optional, Union
 
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
 from airflow.utils.operator_helpers import context_to_airflow_vars
 
 
+def _is_bash_script(input_string: str):
+    input_list = input_string.split(" ")

Review comment:
       Maybe use shlex module?

##########
File path: airflow/operators/bash.py
##########
@@ -157,14 +163,37 @@ def pre_exec():
 
             self.log.info('Running command: %s', self.bash_command)
 
-            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
-                ['bash', "-c", self.bash_command],
-                stdout=PIPE,
-                stderr=STDOUT,
-                cwd=tmp_dir,
-                env=env,
-                preexec_fn=pre_exec,
-            )
+            if isinstance(self.bash_command) == str and not _is_bash_script(self.bash_command):

Review comment:
       ```suggestion
               if isinstance(self.bash_command, str) and _is_bash_script(self.bash_command):
   ```
   

##########
File path: airflow/operators/bash.py
##########
@@ -157,14 +163,37 @@ def pre_exec():
 
             self.log.info('Running command: %s', self.bash_command)
 
-            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
-                ['bash', "-c", self.bash_command],
-                stdout=PIPE,
-                stderr=STDOUT,
-                cwd=tmp_dir,
-                env=env,
-                preexec_fn=pre_exec,
-            )
+            if isinstance(self.bash_command) == str and not _is_bash_script(self.bash_command):
+                warnings.warn(
+                    "Warning: Using a string in the BashOperator leaves your system open to bash injection "
+                    "attacks via escape strings. Please use a list[str] instead."
+                )
+                self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                    ['bash', "-c", self.bash_command],
+                    stdout=PIPE,
+                    stderr=STDOUT,
+                    cwd=tmp_dir,
+                    env=env,
+                    preexec_fn=pre_exec,
+                )
+            elif isinstance(self.bash_command) == list:
+                if (
+                    len(self.bash_command) >= 2
+                    and self.bash_command[0] == "bash"
+                    and self.bash_command[1] == "-c"
+                ):
+                    warnings.warn(
+                        "Warning: using \"bash -c\" for your bash command will give this command"
+                        "full access to the bash environment. Please consider not doing this."

Review comment:
       I don't understand this warning.

##########
File path: airflow/operators/bash.py
##########
@@ -157,14 +163,37 @@ def pre_exec():
 
             self.log.info('Running command: %s', self.bash_command)
 
-            self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
-                ['bash', "-c", self.bash_command],
-                stdout=PIPE,
-                stderr=STDOUT,
-                cwd=tmp_dir,
-                env=env,
-                preexec_fn=pre_exec,
-            )
+            if isinstance(self.bash_command) == str and not _is_bash_script(self.bash_command):
+                warnings.warn(
+                    "Warning: Using a string in the BashOperator leaves your system open to bash injection "
+                    "attacks via escape strings. Please use a list[str] instead."
+                )
+                self.sub_process = Popen(  # pylint: disable=subprocess-popen-preexec-fn
+                    ['bash', "-c", self.bash_command],
+                    stdout=PIPE,
+                    stderr=STDOUT,
+                    cwd=tmp_dir,
+                    env=env,
+                    preexec_fn=pre_exec,
+                )
+            elif isinstance(self.bash_command) == list:
+                if (
+                    len(self.bash_command) >= 2
+                    and self.bash_command[0] == "bash"
+                    and self.bash_command[1] == "-c"
+                ):

Review comment:
       Can't do multi line suggest on mobile app but
   
   ```bash_command[0:2] == ["bash", "-c"]```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] dstandish commented on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
dstandish commented on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778263488


   > Ideas for a better/shorted name appreciated
   
   well `CommandOperator` is shorter if not better :) 
   
   there's also `SubprocessOperator`
   
   or `PopenOperator`


----------------------------------------------------------------
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.

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



[GitHub] [airflow] ashb edited a comment on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
ashb edited a comment on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778297656


   I wonder if we could have a "short" form version that fits in with the taskflow api:
   
   ```python
   @dag
   def my_workflow():
   
       @task
       def munge_output(line):
           return line.upper()
   
       # This is an operator/task that is run, not a static output.
       date = run_command(["date", "--iso=s"])
       munge_output(date)
   ```


----------------------------------------------------------------
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.

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



[GitHub] [airflow] dstandish commented on pull request #12638: Allow BashOperator to take command as list

Posted by GitBox <gi...@apache.org>.
dstandish commented on pull request #12638:
URL: https://github.com/apache/airflow/pull/12638#issuecomment-778264183


   Lastly `ExecOperator`


----------------------------------------------------------------
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.

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