Skip to content

Conversation

@jgao54
Copy link

@jgao54 jgao54 commented Mar 21, 2016

When a task starts, XCom should be cleared for that task instance. (Referencing issue #791)

@landscape-bot
Copy link

Code Health
Code quality remained the same when pulling 7466b76 on jgao54:791 into 9d6b6d5 on airbnb:master.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.009%) to 66.806% when pulling 7466b767f174b42d0d3c83a6369509130210e1d9 on jgao54:791 into 9d6b6d5 on airbnb:master.

@mistercrunch
Copy link
Member

LGTM, @jlowin, as the XCom creator, do you approve?

@jlowin
Copy link
Member

jlowin commented Mar 21, 2016

@jgao54 A common case with XComs is pushing data into the future -- for example, a task creates an XCom when it's run to mark the last_updated time, and checks its own XCom the next time it runs. It looks like this PR would break any sort of self-referential XComs. I think that to properly address issue #791, this needs to be an explicit call (or part of airflow clear), not something run every time.

This is good functionality but my recommendation is to add this function to airflow clear.

@jgao54
Copy link
Author

jgao54 commented Mar 21, 2016

@jlowin good point, overlooked that use case. The next question is should --clear_xcom_data be defaulted to True or False in 'airflow clear'? I think it is more likely to surprise users if XCom data get deleted when all they want to do is clear a TI, thoughts?

@jlowin
Copy link
Member

jlowin commented Mar 21, 2016

@jgao54 I think this is a great idea and something I think I overlooked originally -- as I think through the implementation, this is the wrinkle:

  1. let's say every day a task runs and pushes an XCom
  2. now we clear the task for March 19, 2016
  3. do we really want to clear every XCom that this task has ever created, or just the ones created by the March 19, 2016 task?

I think when an XCom gets pushed the execution date is part of the metadata. How about when you clear a task, it clears any XComs that it created which have the same execution date as the cleared task? Then I would support that being on by default. In the above example, the XCom data pushed on March 19 would disappear but the March 18 data would still be available. What do you think?

@jlowin
Copy link
Member

jlowin commented Mar 21, 2016

Something like this, inside your new function:

session.query(XCom).filter(
    XCom.dag_id == self.dag_id,
    XCom.task_id == self.task_id,
    # this is the new bit, need to check the correct properties
    XCom.execution_date == self.execution_date
).delete()

@jgao54
Copy link
Author

jgao54 commented Mar 21, 2016

@jlowin thanks for the feedback! To me your recommendation is intuitive. According to the original issue #791 it may potentially still comes across as ambiguous to end user. But I think for now it's a good fix.

@jgao54
Copy link
Author

jgao54 commented Mar 21, 2016

  # If the task returns a result, push an XCom containing it
                    if result is not None:
                        self.xcom_push(key=XCOM_RETURN_KEY, value=result)

My guess is we should probably add execution_date here as TI's execution_date if we want to use that approach.

@jlowin
Copy link
Member

jlowin commented Mar 21, 2016

@jgao54 the execution date will be automatically inserted (see https://github.com/airbnb/airflow/blob/master/airflow/models.py#L1293).

Also, after discussion with @r39132, I think my first reaction was too strong. Here's my current thinking:

  • DO clear XComs every time a task is run (as in your original PR)
  • BUT only clear them for the relevant execution date (using the code I pasted above)

@jgao54
Copy link
Author

jgao54 commented Mar 21, 2016

@jlowin Cool, he just synced up with me as well. Will add that line to current change and we should be good to go.

@jgao54
Copy link
Author

jgao54 commented Mar 21, 2016

@jlowin done.

@landscape-bot
Copy link

Code Health
Code quality remained the same when pulling 44f64bb on jgao54:791 into 9d6b6d5 on airbnb:master.

@r39132
Copy link
Contributor

r39132 commented Mar 21, 2016

@jgao54 @jlowin Just making my out-of-band comment in-band for posterity : In the issue, the reporter requested that airflow clear clear the task instance's xcom. I was concerned about the case where someone uses the UI to clear a task instance, that task instance fails (but might have succeeded earlier), and then a downstream task with trigger_rule=All done fires and reads a stale xcom value. Hence, it's best to always clear task_instance side-effects when re-attempted.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.09%) to 66.885% when pulling 44f64bbd464cbe9ed4926dfb2c4d61d467362401 on jgao54:791 into 9d6b6d5 on airbnb:master.

@jlowin
Copy link
Member

jlowin commented Mar 21, 2016

@jgao54 could you please make one more change -- the use of main_session=None is actually outdated. Instead, could you create your function like this:

@provide_session
def clear_xcom_data(self, session=None):
    """
    Clears all XCom data from the database for the task instance
    """
    session.query(XCom).filter(
        XCom.dag_id == self.dag_id,
        XCom.task_id == self.task_id,
        XCom.execution_date == self.execution_date
    ).delete()
    session.commit()

Then you won't need to pass session to clear_xcom_data() when you call it, and you won't need to commit after your call either.

I'm going to make my own PR to clean up some of the other leftover uses of main_session :)
edit: #1188

@landscape-bot
Copy link

Code Health
Code quality remained the same when pulling 34a4095 on jgao54:791 into 9d6b6d5 on airbnb:master.

@landscape-bot
Copy link

Code Health
Code quality remained the same when pulling f238f1d on jgao54:791 into 7dca663 on airbnb:master.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.02%) to 66.898% when pulling f238f1d on jgao54:791 into 7dca663 on airbnb:master.

@coveralls
Copy link

Coverage Status

Coverage increased (+0.02%) to 66.898% when pulling f238f1d on jgao54:791 into 7dca663 on airbnb:master.

r39132 added a commit that referenced this pull request Mar 22, 2016
clear xcom when task starts
@r39132 r39132 merged commit e85770c into apache:master Mar 22, 2016
@rogaha
Copy link

rogaha commented Dec 15, 2016

@jlowin now that this is merged, how is the recommended approach to achieve this scenario:

@jgao54 A common case with XComs is pushing data into the future -- for example, a task creates an XCom when it's run to mark the last_updated time, and checks its own XCom the next time it runs. It looks like this PR would break any sort of self-referential XComs. I think that to properly address issue #791, this needs to be an explicit call (or part of airflow clear), not something run every time.

? My xcom data is deleted on every task run.

@rogaha
Copy link

rogaha commented Dec 15, 2016

/cc @mistercrunch

@jlowin
Copy link
Member

jlowin commented Dec 15, 2016

@rogaha an easy solution is to have two tasks, one dependent on the other, and have the dependent task push the XCom and the first task check for it.

@rogaha
Copy link

rogaha commented Dec 15, 2016

@jlowin I didn't follow. Can you show me an example? How can I pass the latest_updated time between periods (e.g. today vs tomorrow)?

@blrnw3
Copy link

blrnw3 commented Dec 19, 2016

Just FYI, this caused the following bug: https://issues.apache.org/jira/browse/AIRFLOW-703

@jlowin
Copy link
Member

jlowin commented Dec 20, 2016

@rogaha for example, let's say your DAG contains task A and you want task A to check its own last_updated time.

Add a task B to your DAG that fires immediately after task A. All task B does is take A's XCOM and republishes it as its own. That way, the next time A runs, it will look for B's XCOM (which won't have been deleted yet) instead of its own (which will have been deleted).

This should workaround issues related to clearing a task's own XComs.

mobuchowski pushed a commit to mobuchowski/airflow that referenced this pull request Jan 4, 2022
…ache#1180)

* Added BigQueryRelationTransformer and deleted BigQueryNodeVisitor as BigQuery classes aren't accessible from the application classloader at runtime

Signed-off-by: Michael Collado <mike@datakin.com>

* Changed LibraryTest and JDBCRelationVisitorTest to use SparkAgentTestExtension for adding MarquezAgent to runtime so classes are transformed before being loaded

Signed-off-by: Michael Collado <mike@datakin.com>

* Added test for reading from big query and writing to hdfs to validate input and output dataset construction

Signed-off-by: Michael Collado <mike@datakin.com>

* Fixed bug where schema was missing from InsertIntoHadoopFsRelation commands

Signed-off-by: Michael Collado <mike@datakin.com>

* Added fork for every test to avoid BQ class from already being loaded at test runtime

Signed-off-by: Michael Collado <mike@datakin.com>

Co-authored-by: Willy Lulciuc <willy@datakin.com>
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.

8 participants