Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[AIRFLOW-3871] render Operators template fields recursively #4743

Conversation

galak75
Copy link
Contributor

@galak75 galak75 commented Feb 20, 2019

Make sure you have checked all steps below.

Jira

  • My PR addresses the following Airflow Jira issues and references them in the PR title. For example, "[AIRFLOW-XXX] My Airflow PR"

Description

  • Here are some details about my PR, including screenshots of any UI changes:
    see AIRFLOW-3871 Jira ticket description

Tests

  • My PR adds the following unit tests :
    4 additional unit tests in test_python_operator.py

Commits

  • My commits all reference Jira issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "How to write a good git commit message":
    1. Subject is separated from body by a blank line
    2. Subject is limited to 50 characters (not including Jira issue reference)
    3. Subject does not end with a period
    4. Subject uses the imperative mood ("add", not "adding")
    5. Body wraps at 72 characters
    6. Body explains "what" and "why", not "how"

Documentation

  • In case of new functionality, my PR adds documentation that describes how to use it.
    • When adding new operators/hooks/sensors, the autoclass documentation generation needs to be added.
    • All the public functions and the classes in the PR contain docstrings that explain what it does

Code Quality

  • Passes flake8

@galak75 galak75 force-pushed the feature/AIRFLOW-3871-render-template-from-field-recursively branch 2 times, most recently from 0bf4ac0 to a37993b Compare February 22, 2019 16:03
@codecov-io
Copy link

codecov-io commented Feb 22, 2019

Codecov Report

❗ No coverage uploaded for pull request base (master@c098ff7). Click here to learn what that means.
The diff coverage is 11.76%.

Impacted file tree graph

@@           Coverage Diff            @@
##             master   #4743   +/-   ##
========================================
  Coverage          ?   9.47%           
========================================
  Files             ?     607           
  Lines             ?   35032           
  Branches          ?       0           
========================================
  Hits              ?    3321           
  Misses            ?   31711           
  Partials          ?       0
Impacted Files Coverage Δ
airflow/models/baseoperator.py 40.94% <11.76%> (ø)

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update c098ff7...8781b1e. Read the comment docs.

@galak75
Copy link
Contributor Author

galak75 commented Feb 27, 2019

@Fokko : I could not decide between :

  • a recursive template rendering over inner attributes approach (as done in this pull request)
  • a duck typing custom rendering approach (like in this comment)

(see https://issues.apache.org/jira/browse/AIRFLOW-3871)
Is this recursive solution accepted ? is it preferred to a duck typing solution?

setattr(content,
attribute,
self.render_template(attribute, value, context))
except Exception:

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would strongly prefer more granular error handling here. Specifically, any exception raised from render_template should be propagated out.

Copy link
Contributor Author

@galak75 galak75 Mar 11, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My intention was a best effort approach: try to render inner attributes as templates, or else keep attributes unchanged.

e.g: when traversing a UUID attribute, render_template succeeds, but setattr raises an exception (UUID is immutable)

But you have a good point: if my inner template expression is wrong, I would like the exception to be propagated out, so I can understand the issue.
I could just put the render_template call outside try statement?

I will add some test cases right away

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, that would be acceptable. I'd also consider logging any exception at least as a warning.

@bjoernpollex-sc
Copy link

@galak75 Btw., not sure if you've seen this, but I've also commented on the ticket regarding the general approach. I'll repeat here for reference:

I think approach #3 is a bit fragile, for a number of reasons:

  • There might be fields I don't want rendered (maybe I use Jinja templating internally)
  • Due to the dynamic nature of Python, I might want to render fields that can't be found via introspection

I think a viable alternative would be to use the same approach as for operators - declare which fields need templating using a template_fields class variable.

@galak75 galak75 force-pushed the feature/AIRFLOW-3871-render-template-from-field-recursively branch from 122f763 to 24b0004 Compare March 12, 2019 17:59
@galak75
Copy link
Contributor Author

galak75 commented Mar 12, 2019

@bjoernpollex-sc : thanks a lot for your feedback
I actually didn't see your comment on Jira. I do not understand why it did not pop up.

I have some questions about it:


There might be fields I don't want rendered (maybe I use Jinja templating internally)

This is a good point. In such a case, shouldn't Airflow templating process be preferred?


Due to the dynamic nature of Python, I might want to render fields that can't be found via introspection

I'm pretty new with python, how this could happen? Isn't introspection meant to provide all existing attributes, methods, and so on, on an object? Do you have any example or reading about it?


I think a viable alternative would be to use the same approach as for operators - declare which fields need templating using a template_fields class variable.

It is a good alternative to avoid the first point: just choose the fields we want to be templated.
But then, we need to manipulate a class (add a class attribute) to make sure its fields will be rendered.

One thing I really appreciate with approach # 3 is that is works without any change on classes. just setting a templated value on any attribute from any class (in an operator template_field), and this value will be rendered during DAG execution.

If we have to customize a class so that its inner fields are templated, would you rather add a template_field class attribute, or define a render_template custom method (approach # 2)?

I hope I'm clear enough... Thank you in advance for your answer.

@bjoernpollex-sc
Copy link

@galak75 Good points! Responses below:

  1. Airflow templating cannot always be used. For example, we have an operator that generates file-names based on data it pulls from BigQuery. It takes a Jinja-template for the file-name as parameter, and then renders that template many times during execution. There might also be cases of fields that look like Jinja templates, but really aren't.

  2. In Python, objects can implement hooks for attribute access which can for example dynamically generate attributes, or delegate the attribute lookup somewhere else entirely. This is useful when implementing dynamic proxies for example. To be fair though, I can't come up with a realistic use-case for that in this context, so it might not be relevant here.

  3. I'd probably prefer template_fields just for the consistency with operators. I believe it is acceptable to require modifications for classes that want to leverage this, as the way this mechanism works is very specific to begin with. Template attributes are rendered after object initialization, so this only works for attributes that are stored unmodified, and not used in any other way inside __init__. Overall I feel like the potential for subtle bugs with an implicit mechanism is quite high, and as we know from import this: Explicit is better than implicit. ;)

Let me know if you have any more questions, and thanks for putting in the work here!

@galak75
Copy link
Contributor Author

galak75 commented Mar 13, 2019

@bjoernpollex-sc:
Thank you for this constructive discussion. you are bringing very good points to this thread

I totally agree with your statement below

Overall I feel like the potential for subtle bugs with an implicit mechanism is quite high

I have another question though: coming back to the code example in the JIRA description:

Let say that MyAwesomeDataFileTransformer is imported from an external module I cant not change. I guess I could dynamically add the template_fields class attribute before the task declaration? something like:

MyAwesomeDataFileTransformer.template_fields=['input_file', 'output_file']

simple_task = PythonOperator(
    task_id='simple_task',
    provide_context=True,
    python_callable=transform_data_file,
    templates_dict={
	'transformer': MyAwesomeDataFileTransformer(
            "/data/{{ dag.dag_id }}/{{ ts }}/input_file",
            "/data/{{ dag.dag_id }}/{{ ts }}/output_file",
        )
    },
    dag=dag
)

Would it work?

@galak75
Copy link
Contributor Author

galak75 commented Mar 14, 2019

Would it work?

after working on it, it looks like it works (see additional unit tests).

@bjoernpollex-sc: can you please review my latest changes?

@@ -2181,6 +2181,7 @@ def __init__(
# Private attributes
self._upstream_task_ids = set()
self._downstream_task_ids = set()
self._rendered_template_object_ids = set()

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ideally I'd like for this to not be a member. This is only used once, during template rendering, there's no need for having it here. The template rendering method could pass it along as parameter. This is not a big issue, it just seems cleaner to me to limit data to the scope where it's actually needed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would rather keep it as a member to avoid changing BaseOperator.render_template and BaseOperator.render_template_from_field both signatures.

See _render_template_object_recursively implementation

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's a fair point.

@bjoernpollex-sc
Copy link

@galak75 Thanks, this looks very good. The implementation seems fine to me, and the tests seem to cover all the cases. I've left one comment, but I'm not sure if it really needs to be addressed.

Your example of a MyAwesomeDataFileTransformer is actually quite nice to illustrate some of the problems here. For example, what if the __init__-method looks like this:

class MyAwesomeDataFileTransformer:
    def __init__(self, source, target):
        self.source = Path(source)
        self.target = Path(target)

Now, because the paths are wrapped internally in a pathlib.Path instance, you cannot render them anymore (maybe the constructor even raises an exception because these strings aren't valid paths). This doesn't work with any approach. This is why I think that in many cases, classes will have to be customized to work with templating in the first place.

@galak75
Copy link
Contributor Author

galak75 commented Mar 14, 2019

@bjoernpollex-sc, thank you for your feedback.

the statement below works fine:

p = pathlib.Path('/tmp/{{ dag.dag_id }}')

Then, I have an idea to solve the limitation you described: why not add an additional render_template hook in addition to the template_fields one. Then one could define a template_fields list, while another could define a custom render_template method (for cases such the one you described).

Both templating hooks could be used together to provide even more flexibility.

What do you think about it?

@ashb
Copy link
Member

ashb commented Mar 14, 2019

I'm leaning towards a -1 on this - it seems that there are going to be many more case where this won't Just Work leading to confusion on behalf of users.

Can you give a concrete example of when you'd need this? In examples you've shown above why couldn't you create the MyAwesomeDataFileTransformer inside the callable instead?

@bjoernpollex-sc
Copy link

@ashb Suppose you have a GCSFileTransformOperator that applies any given transformation. The problem is that you'll need to duplicate all of the arguments of that transformation on the operator, which needs to propagate them. One way to do this is to have a dictionary for those arguments:

transform = GCSFileTransformOperator(
    source="some/path/{{ds}}/file.xls",
    target="other/path/{{ds}}/file.csv",
    transform=ExcelToCSV,
    transform_args={"quote": True, separator: ","}
)

This works, but what if the operator has multiple such components as inputs? Consider the common file-system interfaces proposed in AIP-14. With those, we can make the operator more generic:

transform = FileTransformOperator(
    source=GCSFileSource,
    source_args={"path": "some/path/{{ds}}/file.xls", "bucket": "some_gcs_bucket", "conn_id": "gcs_conn"),
    target=SFTPFileTarget,
    target_args={"path": "other/path/{{ds}}/file.csv", "conn_id": "sft_conn", "retries": 3},
    transform=ExcelToCSV,
    transform_args={"quote": True, "separator": ","}
)

This starts becoming impractical, it would be much simpler to just write:

transform = FileTransformOperator(
    source=GCSFileSource(path="some/path/{{ds}}/file.xls", bucket="some_gcs_bucket", conn_id="gcs_conn"),
    target=SFTPFileTarget(path="other/path/{{ds}}/file.csv", conn_id="sft_conn", retries=3),
    transform=ExcelToCSV(quote=True, separator=",")
)

This is essentially my use-case in a nutshell. Regarding your concern that this might lead to confusion - that's why I argued for an approach that requires explicit declaration of template fields in all objects/classes that support templates, because that leads to the least surprises (it's explicit, and it follows the current convention for operators). This can easily be added to existing classes, either by monkey-patching or sub-classes, so it's also quite flexible.

@galak75
Copy link
Contributor Author

galak75 commented Mar 15, 2019

I'm leaning towards a -1 on this - it seems that there are going to be many more case where this won't Just Work leading to confusion on behalf of users.

Can you give a concrete example of when you'd need this? In examples you've shown above why couldn't you create the MyAwesomeDataFileTransformer inside the callable instead?

Sure @ashb. Here is the real use case:

We have a lot of different data sources to be imported, transformed and then exported to several destinations. So we tried to decouple our business logic (importing data, transforming it, and then exporting it) from airflow callable functions:

First, we defined a template method to be used as a PythonOperator callable:

def process_data(dataImporter, dataTransformer, dataExporter):
    data = dataImporter.import_data()
    data = dataTransformer.transform_data(data)
    dataExporter.export_data(data)

Then we just have to "inject" the proper data_importer, data_transformer and data_exporter in our DAG task (without having to write a new callable function when the data source is changing or the transformation needs some more parameters...)
let's look at some simple examples:

task1 = PythonOperator(
    task_id='task_1',
    python_callable=process_data,
    op_args=[
        SomeFileDataImporter('/tmp/{{ ds }}/input_data'),
        SomeDataTransformer(some_value, 'path/to/other/file/{{ ds }}/file'),
        SomeFileDataExporter('/data/output/{{ dag.dag_id }}/output_file')
    ],
    dag=dag
)

task2 = PythonOperator(
    task_id='task_2',
    python_callable=process_data,
    op_args=[
        SomeJoiningCsvFilesDataImporter(
            '/tmp/{{ ds }}/input_data_1',
            '/tmp/{{ ds }}/input_data_2',
            join_on='id'
        ),
        SomeOtherDataTransformer(some_value),
        SomeOtherDataExporter(execution_ts={{ ts }})
    ],
    dag=dag
)

This approach has several benefits:

  • separation of concerns: the callable function does not need to know how to import data (it could come from one file, several files, a database, an API, etc...), how to transform it and how to export it.
  • respect single responsibility principle: we separate responsibilities in simple classes: a pickle file importer, a json file importer, a sql importer, etc...
  • use DAG declaration as an IOC tool: we just inject proper implementations, depending on each use case. When another transformation is required, just inject another data_transformer implementation without having to write another callable function. We do not want the callable function being responsible to instantiate the right implementation.

To make this approach work properly, we would like operators to be able to render nested template fields.

The template_fields solution may have some limitation (as pointed out by @bjoernpollex-sc), but it would answer some needed use cases.

I also suggested adding another render_template hook function that people could implement to solve custom template rendering issues. Would it answer your concerns about this submitted pull request?

Thank you for taking time to read this
cheers

@galak75
Copy link
Contributor Author

galak75 commented Mar 19, 2019

Hello @ashb
Do you have any other hint for me to improve this PR ?
Thank you

@galak75 galak75 force-pushed the feature/AIRFLOW-3871-render-template-from-field-recursively branch from d173a12 to 0246619 Compare March 29, 2019 18:02
@galak75
Copy link
Contributor Author

galak75 commented Apr 4, 2019

Hi everyone

I do not want to be rude. I'm sure everyone has a lot to do, especially on this awesome product which is Airflow! But some constructive feedback on this PR (as well on the related issue) would be tremendously appreciated...

About the need: Supporting nested template fields rendering in Operators would be really helpful to better design our DAGs. And it could also probably help other users. Of course, the Airflow team might have some good reason to decide this feature won't be supported. In such a case, we could first have a discussion about the need, and then have a clear explanation why this feature could not be supported.

About the PR: IMHO, the solution reached with @bjoernpollex-sc 's help is solving most of the use cases. Of course, there might be a better solution. Any thoughts or suggestions to improve this PR would be welcome, and I would be glad to rework it if we can find a consensus.

My intent is to help improving Airflow. I'm glad this tool exists and I'm glad I can use it.
Keep up the good work guys!
Thank you

@galak75 galak75 force-pushed the feature/AIRFLOW-3871-render-template-from-field-recursively branch from 0246619 to 80451c1 Compare April 15, 2019 19:31
return result

def _render_nested_template_fields(self, content, context):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since _rendered_template_object_ids is only relevant to this function can we instead change the sig to:

Suggested change
def _render_nested_template_fields(self, content, context):
def _render_nested_template_fields(self, content, context, seen_oids):

Copy link
Contributor Author

@galak75 galak75 May 9, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ashb : I have some concern about this change; I thought about it, but it implies changing BaseOperator.render_template public method signature to propagate theses already seen_oids to nested template fields:

    def render_template(self, attr, content, context):

would become:

    def render_template(self, attr, content, context, seen_oids):

This is why I preferred storing this set as an instance variable.
If you think this is not an issue, just let me know and I'll do it.

(see #4743 (comment))

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this diff should work:

diff --git a/airflow/models/baseoperator.py b/airflow/models/baseoperator.py
index b71f3c4ca1..3e081a5e3d 100644
--- a/airflow/models/baseoperator.py
+++ b/airflow/models/baseoperator.py
@@ -632,7 +633,10 @@ def render_template_from_field(self, attr, content, context, jinja_env):
         Renders a template from a field. If the field is a string, it will
         simply render the string and return the result. If it is a collection or
         nested set of collections, it will traverse the structure and render
-        all elements in it. If the field has another type, it will return it as it is.
+        all elements in it. For any other type, it will recursively render attributes
+        listed in its 'template_fields' attribute (class or instance level attribute)
+        when this 'template_fields' is defined only.
+        Finally returns the rendered field.
         """
         rt = self.render_template
         if isinstance(content, six.string_types):
@@ -644,9 +648,23 @@ def render_template_from_field(self, attr, content, context, jinja_env):
                 k: rt("{}[{}]".format(attr, k), v, context)
                 for k, v in list(content.items())}
         else:
-            result = content
+            result = self._render_nested_template_fields(content, context, seen_oids=set())
         return result
 
+    def _render_nested_template_fields(self, content, context, seen_oids):
+        if id(content) not in seen_oids:
+            seen_oids.add(id(content))
+            try:
+                nested_template_fields = content.template_fields
+            except AttributeError:
+                # content has no inner template fields
+                return content
+
+            for field in nested_template_fields:
+                rendered = self.render_template(field, getattr(content, field), context, seen_oids=seen_oids)
+                setattr(content, field, rendered)
+        return content
+
     def render_template(self, attr, content, context):
         """
         Renders a template either from a file or directly in a field, and returns

But I may have missed something.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @ashb,

I might be wrong, but I think there is one change missing in your diff: render_template signature has to be changed:

            for field in nested_template_fields:
                rendered = self.render_template(field, getattr(content, field), context, seen_oids=seen_oids)
                setattr(content, field, rendered)
        return content

-     def render_template(self, attr, content, context):
+     def render_template(self, attr, content, context, seen_oids):
         """
         Renders a template either from a file or directly in a field, and returns

And this is precisily the change I'm not sure it 's appropriate. What do you think ?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh yeah, I did miss something.

            for field in nested_template_fields:
                rendered = self.render_template(field, getattr(content, field), context, seen_oids=seen_oids)
                setattr(content, field, rendered)
        return content

-     def render_template(self, attr, content, context):
+     def render_template(self, attr, content, context, seen_oids=None):
         """
         Renders a template either from a file or directly in a field, and returns

And then something like this.

                 k: rt("{}[{}]".format(attr, k), v, context)
                 for k, v in list(content.items())}
         else:
-            result = content
+            if seen_oids is None:
+                seen_oids = set()
+            result = self._render_nested_template_fields(content, context, seen_oids=seen_oids)
         return result

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for your feedback.
I understand you would rather change BaseOperator.render_template signature. I will rework this as soon as I can find some time for this (hopefully soon enough!).

@@ -644,9 +648,23 @@ def render_template_from_field(self, attr, content, context, jinja_env):
k: rt("{}[{}]".format(attr, k), v, context)
for k, v in list(content.items())}
else:
result = content
result = self._render_nested_template_fields(content, context)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
result = self._render_nested_template_fields(content, context)
result = self._render_nested_template_fields(content, context, set())

(see next comment)

Copy link
Member

@ashb ashb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Minor changes to the code.

Sorry it sat un-revewied for so long @galak75

This needs adding to the docs/ tree somewhere - probably in https://airflow.apache.org/concepts.html#id1

I missed your first ping, but please @ashb me once you've made these changes. (Oh, though I'm not around for most of May.)

return result

def _render_nested_template_fields(self, content, context):
if id(content) not in self._rendered_template_object_ids:
self._rendered_template_object_ids.add(id(content))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
self._rendered_template_object_ids.add(id(content))
seen_oids.add(id(content))

return result

def _render_nested_template_fields(self, content, context):
if id(content) not in self._rendered_template_object_ids:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
if id(content) not in self._rendered_template_object_ids:
if id(content) not in seen_oids:

return content

for field in nested_template_fields:
rendered = self.render_template(field, getattr(content, field), context)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
rendered = self.render_template(field, getattr(content, field), context)
rendered = self.render_template(field, getattr(content, field), context, seen_oids)

@@ -85,6 +87,10 @@ def setUp(self):
self.addCleanup(self.dag.clear)
self.clear_run()
self.addCleanup(self.clear_run)
try:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Minor point, but since we are making other changes: could you move this to tearDown as it is cleanup code

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@galak75
Copy link
Contributor Author

galak75 commented May 1, 2019

Minor changes to the code.

Sorry it sat un-revewied for so long @galak75

This needs adding to the docs/ tree somewhere - probably in https://airflow.apache.org/concepts.html#id1

I missed your first ping, but please @ashb me once you've made these changes. (Oh, though I'm not around for most of May.)

@ashb : thank you for the review. I'll reworked it pretty soon.

@galak75
Copy link
Contributor Author

galak75 commented May 22, 2019

Hi @ashb

  • I updated the tests as requested;
  • I added some documentation; since my english is far from being fluent, feel free to reword some parts or to suggest some more changes;
  • I have a question about one requested change (see this comment); should I do it despite my concerns? I'm waiting for your opinion here.

Thank you

@bjoernpollex-sc
Copy link

@galak75 Thanks again for the work, hoping to see this in soon! I was wondering though, how does this work with the Rendered view in the web-UI? Would that just call __str__ of the top-level custom object?

@galak75
Copy link
Contributor Author

galak75 commented May 31, 2019

@galak75 Thanks again for the work, hoping to see this in soon! I was wondering though, how does this work with the Rendered view in the web-UI? Would that just call __str__ of the top-level custom object?

Hi @bjoernpollex-sc
I actually did not check the Rendered view. But I guess it should work the exact same way as today: BaseOperator.render_template has not changed... Then if template fields are currently rendered in this Rendered view, they should still be rendered (and nested ones also).

I will try to take a look at this view soon.

@galak75
Copy link
Contributor Author

galak75 commented Jun 10, 2019

Hi @ashb

* I updated the tests as requested;

* I added some documentation; since my english is far from being fluent, feel free to reword some parts or to suggest some more changes;

* I have a question about one requested change (see [this comment](https://github.com/apache/airflow/pull/4743#discussion_r282503945)); should I do it despite my concerns? I'm waiting for your opinion here.

Thank you

Hi @ashb
Will you have a chance to look at this PR ?
thank you

@galak75
Copy link
Contributor Author

galak75 commented Aug 27, 2019

@ashb : ready to be reviewed again!

  1. rebased onto master
  2. addressed your latest comments
  3. migrated all unit tests to TestBaseOperator using the same technique (just calling render_template method directly)

@galak75
Copy link
Contributor Author

galak75 commented Aug 28, 2019

This is a bit awkward: build is failing...
🤔
It succeeded on my fork though... (https://travis-ci.org/VilledeMontreal/incubator-airflow/builds/577525614)

@potiuk
Copy link
Member

potiuk commented Aug 28, 2019 via email

@galak75 galak75 force-pushed the feature/AIRFLOW-3871-render-template-from-field-recursively branch from 7744a10 to 43fbd02 Compare August 29, 2019 19:43
@galak75
Copy link
Contributor Author

galak75 commented Aug 30, 2019

@ashb : any other comments on this PR ?

I think it is ready to be merged
Source code and tests are far simpler after the refactoring that was done on template rendering.

@galak75
Copy link
Contributor Author

galak75 commented Sep 9, 2019

@ashb 🙏

Copy link
Member

@ashb ashb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good @galak75!

It's conflicting with master now though :( Can you ping me when you get a chance to resolve it?

[AIRFLOW-3871] test nested template fields and fix their rendering

[AIRFLOW-3871] test deep nested template fields

[AIRFLOW-3871] test template rendering on circular references

[AIRFLOW-3871] remove unused attribute

[AIRFLOW-3871] add docstring

[AIRFLOW-3871] remove tests now covered in TestBaseOperator class

[AIRFLOW-3871] move test case on missing nested template field

[AIRFLOW-3871] move test case on jinja invalid expression

[AIRFLOW-3871] cleanup PythonOperator tests

[AIRFLOW-3871] add test case with None value on nested template field

[AIRFLOW-3871] add test cases with None value and empty collections

[AIRFLOW-3871] refactor nested field rendering

[AIRFLOW-3871] extract function to actually render template fields

[AIRFLOW-3871] reuse extracted function for top fields rendering

[AIRFLOW-3871] format code

[AIRFLOW-3871] add typing definition to private method

[AIRFLOW-3871] nested template field rendering now returns nothing

[AIRFLOW-3871] nested template field rendering now returns nothing
@galak75 galak75 force-pushed the feature/AIRFLOW-3871-render-template-from-field-recursively branch from 43fbd02 to 8781b1e Compare September 17, 2019 14:22
@galak75
Copy link
Contributor Author

galak75 commented Sep 17, 2019

@ashb: I rebased onto master and build is green

@ashb ashb merged commit d567f9a into apache:master Sep 18, 2019
@ashb
Copy link
Member

ashb commented Sep 18, 2019

Sorry it took so many months to get this in, and thanks for sticking with us!

@galak75
Copy link
Contributor Author

galak75 commented Sep 18, 2019

Sorry it took so many months to get this in, and thanks for sticking with us!

Sometimes, things don't go as smoothly as we want...
I'm happy this PR has been merged!
Thank you
🎉

@galak75 galak75 deleted the feature/AIRFLOW-3871-render-template-from-field-recursively branch September 18, 2019 14:17
@bjoernpollex-sc
Copy link

Same here, this is really awesome, can't wait for the next release!

ashb pushed a commit that referenced this pull request Sep 24, 2019
ashb pushed a commit to ashb/airflow that referenced this pull request Sep 25, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants