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

BigQuery insert_overwrite incremental strategy fails for day partitioned tables #3095

Closed
1 of 5 tasks
osusam28 opened this issue Feb 11, 2021 · 6 comments · Fixed by #3098
Closed
1 of 5 tasks

BigQuery insert_overwrite incremental strategy fails for day partitioned tables #3095

osusam28 opened this issue Feb 11, 2021 · 6 comments · Fixed by #3098
Labels
bigquery bug Something isn't working regression

Comments

@osusam28
Copy link

Description

I have a config for a model (using BigQuery) like below (in the model file):

{{
  config(
    partition_by = {
        'field': 'ETL_CREATE_DTS',
        'data_type': 'timestamp',
        'granularity': 'day'
    },
    unique_key = "TRANSACTIONID",
    incremental_strategy = 'insert_overwrite'
  )
}}

ETL_CREATE_DTS is of the type TIMESTAMP

When running this model the first time (dbt run), it succeeds. However the second time, when the query is referencing the target table to find out what partitions to overwrite, it fails with the error:

Database Error in model msa_premium_transaction_fact_clean (models/cleaning/msa/msa_premium_transaction_fact_clean.sql)
  Query error: Cannot coerce expression (
            select as struct
                array_agg(distinct timestamp_trunc(ETL_CREATE_DTS, day))
            from `gcp-ent-property-etl-dev`.`sample_dw`.`msa_premium_transaction_fact_clean__dbt_tmp`
        ) to type STRUCT<ARRAY<DATE>> at [106:46]
  compiled SQL at target/run/dbt_etl_poc/models/cleaning/msa/msa_premium_transaction_fact_clean.sql

Turns out the generated SQL for this insert_overwrite is incorrect:

-- generated script to merge partitions into `gcp-ent-property-etl-dev`.`sample_dw`.`msa_premium_transaction_fact_clean`
      declare dbt_partitions_for_replacement array<date>;
      declare _dbt_max_partition timestamp;

      set _dbt_max_partition = (
          select max(ETL_CREATE_DTS) from `gcp-ent-property-etl-dev`.`sample_dw`.`msa_premium_transaction_fact_clean`
      );

      -- 1. create a temp table

....
      
      -- 2. define partitions to update
      set (dbt_partitions_for_replacement) = (
          select as struct
              array_agg(distinct timestamp_trunc(ETL_CREATE_DTS, day))
          from `project-id`.`sample_dw`.`msa_premium_transaction_fact_clean__dbt_tmp`
      );

      
      -- 3. run the merge statement
...

Notice the variable declare dbt_partitions_for_replacement array<date>; will not match the output of (2) because timestamp_trunc returns a TIMESTAMP type.

Steps To Reproduce

In as much detail as possible, please provide steps to reproduce the issue. Sample data that triggers the issue, example model code, etc is all very helpful here.

Expected behavior

I should be able to declare a BigQuery table partitioned by day on a timestamp field, and the insert_overwrite strategy work.
A quick fix in the code seems to be the declared variable needs to have the same datatype defined for the array as the data type of the partitioned field in the BigQuery table (see below)

It looks as if this line (https:/fishtown-analytics/dbt/blob/2b48152da66dbd7f07272983bbc261f1b6924f20/plugins/bigquery/dbt/include/bigquery/macros/materializations/incremental.sql#L20) could be removed to allow the the variable definition (https:/fishtown-analytics/dbt/blob/2b48152da66dbd7f07272983bbc261f1b6924f20/plugins/bigquery/dbt/include/bigquery/macros/materializations/incremental.sql#L52) to reflect the exact data type.

System information

Which database are you using dbt with?

  • postgres
  • redshift
  • bigquery
  • snowflake
  • other (specify: ____________)

The output of dbt --version:

installed version: 0.19.0
   latest version: 0.19.0

Up to date!

Plugins:
  - bigquery: 0.19.0
  - snowflake: 0.19.0
  - redshift: 0.19.0
  - postgres: 0.19.0

The operating system you're using:
macOS Catalina

The output of python --version:
Python 2.7.16

@osusam28 osusam28 added bug Something isn't working triage labels Feb 11, 2021
@noahbruegmann
Copy link

We've experienced this same behavior. I'd note that this isn't a problem in 0.18.1., where the query used to create the struct returned dates rather than timestamps.

@jtcohen6
Copy link
Contributor

Thanks for the report @osusam28 @noahbruegmann!

This is a regression in v0.19.0, an unintended side-effect of the addition of granularity. As you noted, the fix here should be quite straightforward: BigQuery now supports timestamp-type partitions, so there's no need to cast to date as before. This may require a one-time full-refresh for folks who have current incremental models partitioned by a date type (date(timestamp_col)) who will now be partitioning by a timestamp type (timestamp_trunc(timestamp_col, day)).

In any case, we'll get this fixed for v0.19.1. We also need better test coverage for the matrix of potential partition data types and incremental strategies, to avoid regressions like this one (or #3063) from happening in the future.

@jtcohen6
Copy link
Contributor

Great news: BigQuery considers a table to be partitioned by Day, regardless of whether that's date_col, date(timestamp_col), or timestamp_trunc(timestamp_col, day). That's a pleasant surprise, and it means that both incremental updates and full refreshes (via create or replace, without need for hard drop-and-recreate) will work just fine when upgrading from v0.18.1 → v0.19.1.

I'll work on a PR for this now.

@funnel-arvid
Copy link

funnel-arvid commented Feb 18, 2021

Don't know if this is related, but "granularity" fails in Dbt Cloud. Why is that?

Could not parse partition config: at path []: Additional properties are not allowed ('granularity' was unexpected)

@jtcohen6
Copy link
Contributor

@funnel-arvid Are you using dbt v0.19.0? Could you share the snippet of code (in dbt_project.yml, it sounds like)?

@funnel-arvid
Copy link

@jtcohen6 Oh gosh, I just now realised you can choose and change the dbt version in cloud under your environment. I was on 0.17.0. Sorry for nothing!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bigquery bug Something isn't working regression
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants