dbt-labs / dbt-core

dbt enables data analysts and engineers to transform their data using the same practices that software engineers use to build applications.
https://getdbt.com
Apache License 2.0
9.6k stars 1.59k forks source link

[CT-2380] [Bug] Referencing Ephemeral SQL Models Fails in Python #7288

Open ernestoongaro opened 1 year ago

ernestoongaro commented 1 year ago

Is this a new bug in dbt-core?

Current Behavior

Doing this writes the SQL directly into the Python file and fails fairly miserably

def model(dbt, session):

    ephemeral_df = dbt.ref("my_ephemeral_sql_model")

    return ephemeral_df

Expected Behavior

Ideally it would be good support this, or give a compilation error vs trying to send invalid python down

Steps To Reproduce

  1. Write a simple SQL ephemeral model, with select 1 as test, save it as my_ephemeral_sql_model.sql
  2. Write this python file

    def model(dbt, session):
    
    orders_df = dbt.ref("my_ephemeral_sql_model")
    
    return orders_df

Relevant log output

WITH ernie__dbt_sp AS PROCEDURE ()

RETURNS STRING
LANGUAGE PYTHON
RUNTIME_VERSION = '3.8'
PACKAGES = ('snowflake-snowpark-python')

HANDLER = 'main'
EXECUTE AS CALLER
AS
$$

import sys
sys._xoptions['snowflake_partner_attribution'].append("dbtLabs_dbtPython")

with __dbt__cte__stg_orders as (

select 
o_orderkey as order_id,
o_custkey as customer_id,
o_totalprice as price_paid
from raw.tpch_sf001.orders
)def model(dbt, session):

    orders_df = dbt.ref("stg_orders")

    return orders_df

# This part is user provided model code
# you will need to copy the next section to run the code
# COMMAND ----------
# this part is dbt logic for get ref work, do not modify

def ref(*args,dbt_load_df_function):
    refs = {"stg_orders": "__dbt__cte__stg_orders"}
    key = '.'.join(args)
    return dbt_load_df_function(refs[key])

def source(*args, dbt_load_df_function):
    sources = {}
    key = '.'.join(args)
    return dbt_load_df_function(sources[key])

config_dict = {}

class config:
    def __init__(self, *args, **kwargs):
        pass

    @staticmethod
    def get(key, default=None):
        return config_dict.get(key, default)

class this:
    """dbt.this() or dbt.this.identifier"""
    database = "analytics"
    schema = "dbt_eongaro"
    identifier = "ernie"

    def __repr__(self):
        return 'analytics.dbt_eongaro.ernie'

class dbtObj:
    def __init__(self, load_df_function) -> None:
        self.source = lambda *args: source(*args, dbt_load_df_function=load_df_function)
        self.ref = lambda *args: ref(*args, dbt_load_df_function=load_df_function)
        self.config = config
        self.this = this()
        self.is_incremental = False

# COMMAND ----------

# To run this in snowsight, you need to select entry point to be main
# And you may have to modify the return type to text to get the result back
# def main(session):
#     dbt = dbtObj(session.table)
#     df = model(dbt, session)
#     return df.collect()

# to run this in local notebook, you need to create a session following examples https://github.com/Snowflake-Labs/sfguide-getting-started-snowpark-python
# then you can do the following to run model
# dbt = dbtObj(session.table)
# df = model(dbt, session)

def materialize(session, df, target_relation):
    # make sure pandas exists
    import importlib.util
    package_name = 'pandas'
    if importlib.util.find_spec(package_name):
        import pandas
        if isinstance(df, pandas.core.frame.DataFrame):
          # session.write_pandas does not have overwrite function
          df = session.createDataFrame(df)
    df.write.mode("overwrite").save_as_table("analytics.dbt_eongaro.ernie", create_temp_table=False)

def main(session):
    dbt = dbtObj(session.table)
    df = model(dbt, session)
    materialize(session, df, dbt.this)
    return "OK"

$$
CALL ernie__dbt_sp();
13:44:00  Opening a new connection, currently in state closed
13:44:21  Snowflake adapter: Snowflake query id: 01ab7258-0502-c699-0004-7d83098783fa
13:44:21  Snowflake adapter: Snowflake error: 100357 (P0000): Python Interpreter Error:
  File "_udf_code.py", line 13
    o_orderkey as order_id,
    ^
SyntaxError: invalid syntax
 in function ERNIE__DBT_SP with handler main
13:44:21  Timing info for model.tpch.ernie (execute): 2023-04-06 13:44:00.493765 => 2023-04-06 13:44:21.430217
13:44:21  On model.tpch.ernie: Close
13:44:21  Database Error in model ernie (models/dbt101/ernie.py)
  100357 (P0000): Python Interpreter Error:
    File "_udf_code.py", line 13
      o_orderkey as order_id,
      ^
  SyntaxError: invalid syntax
   in function ERNIE__DBT_SP with handler main
  compiled Code at target/run/tpch/models/dbt101/ernie.py
13:44:21  Sending event: {'category': 'dbt', 'action': 'run_model', 'label': '1d89b487-c96c-4541-a6ed-fab238fdac26', 'context': [<snowplow_tracker.self_describing_json.SelfDescribingJson object at 0x7f706f42eac0>]}
13:44:21  1 of 1 ERROR creating python table model dbt_eongaro.ernie ..................... [ERROR in 21.36s]
13:44:21  Finished running node model.tpch.ernie

Environment

- OS: dbt Cloud
- dbt: 1.4 latest

Which database adapter are you using with dbt?

snowflake

Additional Context

No response

dbeatty10 commented 1 year ago

Thanks for reporting this @ernestoongaro !

I got the same results as you when I tried the same thing in dbt-duckdb.

There's two main options for resolving this:

  1. Update dbt-core so that dbt python models can reference ephemeral sql models
  2. Raise an error if a dbt python model tries to reference an ephemeral sql model

Option 1

I'm not sure if there's an easy path to support referencing ephemeral nodes from dbt python models or not. Lots of juicy details in the Compiler class of how ephemeral models are injected as a CTE into SQL models (and apparently Python models as well 😬 ).

Option 2

If it's non-trivial to add support for Python, one idea would be to proactively raise as an error in the meantime to say that referencing ephemeral models isn't supported.

Maybe add something like this somewhere around here:

if node.language == ModelLanguage.python:
    raise DbtInternalError(
        f"During compilation, found an ephemeral model named {cte.id}, but ephemeral models are not supported for Python."
    )

Of course we'd want to add an explanation within docs.getdbt.com as well if we choose Option 2.

jtcohen6 commented 1 year ago

Option 2 makes sense to me in the shorter term!

We do something similar within the run-operation task today (which can't handle direct references to ephemeral models because of ... honestly, let's not get into it):

https://github.com/dbt-labs/dbt-core/blob/fe62ab8ec5df334985505bf03c5dd10b465ef995/core/dbt/context/providers.py#L495-L510

I don't think it would make sense to have a custom RefResolver just for Python models, but we could add some conditional logic to RuntimeRefResolver that checks for self.language and target_model.is_ephemeral_model.

jtcohen6 commented 1 year ago

PR to update the docs (thanks @ernestoongaro!!)

github-actions[bot] commented 7 months ago

This issue has been marked as Stale because it has been open for 180 days with no activity. If you would like the issue to remain open, please comment on the issue or else it will be closed in 7 days.

github-actions[bot] commented 7 months ago

Although we are closing this issue as stale, it's not gone forever. Issues can be reopened if there is renewed community interest. Just add a comment to notify the maintainers.

dbeatty10 commented 5 months ago

Marking as un-stale because there's an open PR and it also has a recent comment: https://github.com/dbt-labs/dbt-core/pull/9196#issuecomment-1923322305